diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index 6eccb65344..6d48859430 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -202,6 +202,7 @@ jobs: value: | org.apache.comet.exec.CometShuffleSuite org.apache.comet.exec.CometShuffle4_0Suite + org.apache.comet.exec.CometNativeColumnarToRowSuite org.apache.comet.exec.CometNativeShuffleSuite org.apache.comet.exec.CometShuffleEncryptionSuite org.apache.comet.exec.CometShuffleManagerSuite diff --git a/.github/workflows/pr_build_macos.yml b/.github/workflows/pr_build_macos.yml index d76edc008d..60bc8ca36b 100644 --- a/.github/workflows/pr_build_macos.yml +++ b/.github/workflows/pr_build_macos.yml @@ -145,6 +145,7 @@ jobs: value: | org.apache.comet.exec.CometShuffleSuite org.apache.comet.exec.CometShuffle4_0Suite + org.apache.comet.exec.CometNativeColumnarToRowSuite org.apache.comet.exec.CometNativeShuffleSuite org.apache.comet.exec.CometShuffleEncryptionSuite org.apache.comet.exec.CometShuffleManagerSuite diff --git a/.gitignore b/.gitignore index 9978e37bdf..05b37627bd 100644 --- a/.gitignore +++ b/.gitignore @@ -1,3 +1,4 @@ +CLAUDE.md target .idea *.iml diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 656dbc9a58..efb36325e9 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -296,6 +296,17 @@ object CometConf extends ShimCometConf { val COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("localTableScan", defaultValue = false) + val COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED: ConfigEntry[Boolean] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.columnarToRow.native.enabled") + .category(CATEGORY_EXEC) + .doc( + "Whether to enable native columnar to row conversion. When enabled, Comet will use " + + "native Rust code to convert Arrow columnar data to Spark UnsafeRow format instead " + + "of the JVM implementation. This can improve performance for queries that need to " + + "convert between columnar and row formats.") + .booleanConf + .createWithDefault(true) + val COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.exec.sortMergeJoinWithJoinFilter.enabled") .category(CATEGORY_ENABLE_EXEC) diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala index d5d6ded553..45245121a0 100644 --- a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala +++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala @@ -78,6 +78,26 @@ class NativeUtil { (arrays, schemas) } + /** + * Exports a ColumnarBatch to Arrow FFI and returns the memory addresses. + * + * This is a convenience method that allocates Arrow structs, exports the batch, and returns + * just the memory addresses (without exposing the Arrow types). + * + * @param batch + * the columnar batch to export + * @return + * a tuple of (array addresses, schema addresses, number of rows) + */ + def exportBatchToAddresses(batch: ColumnarBatch): (Array[Long], Array[Long], Int) = { + val numCols = batch.numCols() + val (arrays, schemas) = allocateArrowStructs(numCols) + val arrayAddrs = arrays.map(_.memoryAddress()) + val schemaAddrs = schemas.map(_.memoryAddress()) + val numRows = exportBatch(arrayAddrs, schemaAddrs, batch) + (arrayAddrs, schemaAddrs, numRows) + } + /** * Exports a Comet `ColumnarBatch` into a list of memory addresses that can be consumed by the * native execution. diff --git a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala index 036c5c9aaf..c7dc82c1b9 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala +++ b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala @@ -26,7 +26,7 @@ import java.nio.channels.Channels import scala.jdk.CollectionConverters._ import org.apache.arrow.c.CDataDictionaryProvider -import org.apache.arrow.vector.{BigIntVector, BitVector, DateDayVector, DecimalVector, FieldVector, FixedSizeBinaryVector, Float4Vector, Float8Vector, IntVector, SmallIntVector, TimeStampMicroTZVector, TimeStampMicroVector, TinyIntVector, ValueVector, VarBinaryVector, VarCharVector, VectorSchemaRoot} +import org.apache.arrow.vector.{BigIntVector, BitVector, DateDayVector, DecimalVector, FieldVector, FixedSizeBinaryVector, Float4Vector, Float8Vector, IntVector, NullVector, SmallIntVector, TimeStampMicroTZVector, TimeStampMicroVector, TinyIntVector, ValueVector, VarBinaryVector, VarCharVector, VectorSchemaRoot} import org.apache.arrow.vector.complex.{ListVector, MapVector, StructVector} import org.apache.arrow.vector.dictionary.DictionaryProvider import org.apache.arrow.vector.ipc.ArrowStreamWriter @@ -282,7 +282,7 @@ object Utils extends CometTypeShim { _: BigIntVector | _: Float4Vector | _: Float8Vector | _: VarCharVector | _: DecimalVector | _: DateDayVector | _: TimeStampMicroTZVector | _: VarBinaryVector | _: FixedSizeBinaryVector | _: TimeStampMicroVector | _: StructVector | _: ListVector | - _: MapVector) => + _: MapVector | _: NullVector) => v.asInstanceOf[FieldVector] case _ => throw new SparkException(s"Unsupported Arrow Vector for $reason: ${valueVector.getClass}") diff --git a/native/core/src/execution/columnar_to_row.rs b/native/core/src/execution/columnar_to_row.rs new file mode 100644 index 0000000000..a1a942de95 --- /dev/null +++ b/native/core/src/execution/columnar_to_row.rs @@ -0,0 +1,3128 @@ +// 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. + +//! Native implementation of columnar to row conversion for Spark UnsafeRow format. +//! +//! This module converts Arrow columnar data to Spark's UnsafeRow format, which is used +//! for row-based operations in Spark. The conversion is done in native code for better +//! performance compared to the JVM implementation. +//! +//! # UnsafeRow Format +//! +//! ```text +//! ┌─────────────────────────────────────────────────────────────┐ +//! │ Null Bitset: ((numFields + 63) / 64) * 8 bytes │ +//! ├─────────────────────────────────────────────────────────────┤ +//! │ Fixed-width portion: 8 bytes per field │ +//! │ - Primitives: value stored directly (in lowest bytes) │ +//! │ - Variable-length: (offset << 32) | length │ +//! ├─────────────────────────────────────────────────────────────┤ +//! │ Variable-length data: 8-byte aligned │ +//! └─────────────────────────────────────────────────────────────┘ +//! ``` + +use crate::errors::{CometError, CometResult}; +use arrow::array::types::{ + ArrowDictionaryKeyType, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, UInt32Type, + UInt64Type, UInt8Type, +}; +use arrow::array::*; +use arrow::compute::{cast_with_options, CastOptions}; +use arrow::datatypes::{ArrowNativeType, DataType, TimeUnit}; +use std::sync::Arc; + +/// Maximum digits for decimal that can fit in a long (8 bytes). +const MAX_LONG_DIGITS: u8 = 18; + +/// Pre-downcast array reference to avoid type dispatch in inner loops. +/// This enum holds references to concrete array types, allowing direct access +/// without repeated downcast_ref calls. +enum TypedArray<'a> { + Null, + Boolean(&'a BooleanArray), + Int8(&'a Int8Array), + Int16(&'a Int16Array), + Int32(&'a Int32Array), + Int64(&'a Int64Array), + Float32(&'a Float32Array), + Float64(&'a Float64Array), + Date32(&'a Date32Array), + TimestampMicro(&'a TimestampMicrosecondArray), + Decimal128(&'a Decimal128Array, u8), // array + precision + String(&'a StringArray), + LargeString(&'a LargeStringArray), + Binary(&'a BinaryArray), + LargeBinary(&'a LargeBinaryArray), + FixedSizeBinary(&'a FixedSizeBinaryArray), + Struct( + &'a StructArray, + arrow::datatypes::Fields, + Vec>, + ), + List(&'a ListArray, arrow::datatypes::FieldRef), + LargeList(&'a LargeListArray, arrow::datatypes::FieldRef), + Map(&'a MapArray, arrow::datatypes::FieldRef), + Dictionary(&'a ArrayRef, DataType), // fallback for dictionary types +} + +impl<'a> TypedArray<'a> { + /// Pre-downcast an ArrayRef to a TypedArray. + fn from_array(array: &'a ArrayRef, _schema_type: &DataType) -> CometResult { + let actual_type = array.data_type(); + match actual_type { + DataType::Null => { + // Verify the array is actually a NullArray, but we don't need to store the reference + // since all values are null by definition + if array.as_any().downcast_ref::().is_none() { + return Err(CometError::Internal(format!( + "Failed to downcast to NullArray, actual type: {:?}", + array.data_type() + ))); + } + Ok(TypedArray::Null) + } + DataType::Boolean => Ok(TypedArray::Boolean( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to BooleanArray, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Int8 => Ok(TypedArray::Int8( + array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int8Array, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Int16 => Ok(TypedArray::Int16( + array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int16Array, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Int32 => Ok(TypedArray::Int32( + array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int32Array, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Int64 => Ok(TypedArray::Int64( + array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int64Array, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Float32 => Ok(TypedArray::Float32( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Float32Array, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Float64 => Ok(TypedArray::Float64( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Float64Array, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Date32 => Ok(TypedArray::Date32( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Date32Array, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Timestamp(TimeUnit::Microsecond, _) => Ok(TypedArray::TimestampMicro( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to TimestampMicrosecondArray, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Decimal128(p, _) => Ok(TypedArray::Decimal128( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Decimal128Array, actual type: {:?}", + array.data_type() + )) + })?, + *p, + )), + DataType::Utf8 => Ok(TypedArray::String( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to StringArray, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::LargeUtf8 => Ok(TypedArray::LargeString( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to LargeStringArray, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Binary => Ok(TypedArray::Binary( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to BinaryArray, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::LargeBinary => Ok(TypedArray::LargeBinary( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to LargeBinaryArray, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::FixedSizeBinary(_) => Ok(TypedArray::FixedSizeBinary( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to FixedSizeBinaryArray, actual type: {:?}", + array.data_type() + )) + })?, + )), + DataType::Struct(fields) => { + let struct_arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to StructArray, actual type: {:?}", + array.data_type() + )) + })?; + // Pre-downcast all struct fields once + let typed_fields: Vec = fields + .iter() + .enumerate() + .map(|(idx, field)| { + TypedElements::from_array(struct_arr.column(idx), field.data_type()) + }) + .collect(); + Ok(TypedArray::Struct(struct_arr, fields.clone(), typed_fields)) + } + DataType::List(field) => Ok(TypedArray::List( + array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to ListArray, actual type: {:?}", + array.data_type() + )) + })?, + Arc::clone(field), + )), + DataType::LargeList(field) => Ok(TypedArray::LargeList( + array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to LargeListArray, actual type: {:?}", + array.data_type() + )) + })?, + Arc::clone(field), + )), + DataType::Map(field, _) => Ok(TypedArray::Map( + array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to MapArray, actual type: {:?}", + array.data_type() + )) + })?, + Arc::clone(field), + )), + DataType::Dictionary(_, _) => Ok(TypedArray::Dictionary(array, actual_type.clone())), + _ => Err(CometError::Internal(format!( + "Unsupported data type for pre-downcast: {:?}", + actual_type + ))), + } + } + + /// Check if the value at the given index is null. + #[inline] + fn is_null(&self, row_idx: usize) -> bool { + match self { + TypedArray::Null => true, // Null type is always null + TypedArray::Boolean(arr) => arr.is_null(row_idx), + TypedArray::Int8(arr) => arr.is_null(row_idx), + TypedArray::Int16(arr) => arr.is_null(row_idx), + TypedArray::Int32(arr) => arr.is_null(row_idx), + TypedArray::Int64(arr) => arr.is_null(row_idx), + TypedArray::Float32(arr) => arr.is_null(row_idx), + TypedArray::Float64(arr) => arr.is_null(row_idx), + TypedArray::Date32(arr) => arr.is_null(row_idx), + TypedArray::TimestampMicro(arr) => arr.is_null(row_idx), + TypedArray::Decimal128(arr, _) => arr.is_null(row_idx), + TypedArray::String(arr) => arr.is_null(row_idx), + TypedArray::LargeString(arr) => arr.is_null(row_idx), + TypedArray::Binary(arr) => arr.is_null(row_idx), + TypedArray::LargeBinary(arr) => arr.is_null(row_idx), + TypedArray::FixedSizeBinary(arr) => arr.is_null(row_idx), + TypedArray::Struct(arr, _, _) => arr.is_null(row_idx), + TypedArray::List(arr, _) => arr.is_null(row_idx), + TypedArray::LargeList(arr, _) => arr.is_null(row_idx), + TypedArray::Map(arr, _) => arr.is_null(row_idx), + TypedArray::Dictionary(arr, _) => arr.is_null(row_idx), + } + } + + /// Get the fixed-width value as i64 (for types that fit in 8 bytes). + #[inline] + fn get_fixed_value(&self, row_idx: usize) -> i64 { + match self { + TypedArray::Boolean(arr) => { + if arr.value(row_idx) { + 1i64 + } else { + 0i64 + } + } + TypedArray::Int8(arr) => arr.value(row_idx) as i64, + TypedArray::Int16(arr) => arr.value(row_idx) as i64, + TypedArray::Int32(arr) => arr.value(row_idx) as i64, + TypedArray::Int64(arr) => arr.value(row_idx), + TypedArray::Float32(arr) => arr.value(row_idx).to_bits() as i64, + TypedArray::Float64(arr) => arr.value(row_idx).to_bits() as i64, + TypedArray::Date32(arr) => arr.value(row_idx) as i64, + TypedArray::TimestampMicro(arr) => arr.value(row_idx), + TypedArray::Decimal128(arr, precision) => { + if *precision <= MAX_LONG_DIGITS { + arr.value(row_idx) as i64 + } else { + 0 // Variable-length decimal, handled elsewhere + } + } + // Variable-length types return 0, actual value written separately + _ => 0, + } + } + + /// Check if this is a variable-length type. + #[inline] + fn is_variable_length(&self) -> bool { + match self { + TypedArray::Null + | TypedArray::Boolean(_) + | TypedArray::Int8(_) + | TypedArray::Int16(_) + | TypedArray::Int32(_) + | TypedArray::Int64(_) + | TypedArray::Float32(_) + | TypedArray::Float64(_) + | TypedArray::Date32(_) + | TypedArray::TimestampMicro(_) => false, + TypedArray::Decimal128(_, precision) => *precision > MAX_LONG_DIGITS, + _ => true, + } + } + + /// Write variable-length data to buffer. Returns actual length (0 if not variable-length). + fn write_variable_to_buffer(&self, buffer: &mut Vec, row_idx: usize) -> CometResult { + match self { + TypedArray::String(arr) => { + let bytes = arr.value(row_idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedArray::LargeString(arr) => { + let bytes = arr.value(row_idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedArray::Binary(arr) => { + let bytes = arr.value(row_idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedArray::LargeBinary(arr) => { + let bytes = arr.value(row_idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedArray::FixedSizeBinary(arr) => { + let bytes = arr.value(row_idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedArray::Decimal128(arr, precision) if *precision > MAX_LONG_DIGITS => { + let bytes = i128_to_spark_decimal_bytes(arr.value(row_idx)); + let len = bytes.len(); + buffer.extend_from_slice(&bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedArray::Struct(arr, fields, typed_fields) => { + write_struct_to_buffer_typed(buffer, arr, row_idx, fields, typed_fields) + } + TypedArray::List(arr, field) => write_list_to_buffer(buffer, arr, row_idx, field), + TypedArray::LargeList(arr, field) => { + write_large_list_to_buffer(buffer, arr, row_idx, field) + } + TypedArray::Map(arr, field) => write_map_to_buffer(buffer, arr, row_idx, field), + TypedArray::Dictionary(arr, schema_type) => { + if let DataType::Dictionary(key_type, value_type) = schema_type { + write_dictionary_to_buffer( + buffer, + arr, + row_idx, + key_type.as_ref(), + value_type.as_ref(), + ) + } else { + Err(CometError::Internal(format!( + "Expected Dictionary type but got {:?}", + schema_type + ))) + } + } + _ => Ok(0), // Fixed-width types + } + } +} + +/// Pre-downcast element array for list/array types. +/// This allows direct access to element values without per-row allocation. +enum TypedElements<'a> { + Boolean(&'a BooleanArray), + Int8(&'a Int8Array), + Int16(&'a Int16Array), + Int32(&'a Int32Array), + Int64(&'a Int64Array), + Float32(&'a Float32Array), + Float64(&'a Float64Array), + Date32(&'a Date32Array), + TimestampMicro(&'a TimestampMicrosecondArray), + Decimal128(&'a Decimal128Array, u8), + String(&'a StringArray), + LargeString(&'a LargeStringArray), + Binary(&'a BinaryArray), + LargeBinary(&'a LargeBinaryArray), + FixedSizeBinary(&'a FixedSizeBinaryArray), + // For nested types, fall back to ArrayRef + Other(&'a ArrayRef, DataType), +} + +impl<'a> TypedElements<'a> { + /// Create from an ArrayRef and element type. + fn from_array(array: &'a ArrayRef, element_type: &DataType) -> Self { + match element_type { + DataType::Boolean => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Boolean(arr); + } + } + DataType::Int8 => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Int8(arr); + } + } + DataType::Int16 => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Int16(arr); + } + } + DataType::Int32 => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Int32(arr); + } + } + DataType::Int64 => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Int64(arr); + } + } + DataType::Float32 => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Float32(arr); + } + } + DataType::Float64 => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Float64(arr); + } + } + DataType::Date32 => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Date32(arr); + } + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::TimestampMicro(arr); + } + } + DataType::Decimal128(p, _) => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Decimal128(arr, *p); + } + } + DataType::Utf8 => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::String(arr); + } + } + DataType::LargeUtf8 => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::LargeString(arr); + } + } + DataType::Binary => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Binary(arr); + } + } + DataType::LargeBinary => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::LargeBinary(arr); + } + } + DataType::FixedSizeBinary(_) => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::FixedSizeBinary(arr); + } + } + _ => {} + } + TypedElements::Other(array, element_type.clone()) + } + + /// Get element size for UnsafeArrayData format. + fn element_size(&self) -> usize { + match self { + TypedElements::Boolean(_) => 1, + TypedElements::Int8(_) => 1, + TypedElements::Int16(_) => 2, + TypedElements::Int32(_) | TypedElements::Date32(_) | TypedElements::Float32(_) => 4, + TypedElements::Int64(_) + | TypedElements::TimestampMicro(_) + | TypedElements::Float64(_) => 8, + TypedElements::Decimal128(_, p) if *p <= MAX_LONG_DIGITS => 8, + _ => 8, // Variable-length uses 8 bytes for offset+length + } + } + + /// Check if this is a fixed-width primitive type that supports bulk copy. + fn supports_bulk_copy(&self) -> bool { + matches!( + self, + TypedElements::Int8(_) + | TypedElements::Int16(_) + | TypedElements::Int32(_) + | TypedElements::Int64(_) + | TypedElements::Float32(_) + | TypedElements::Float64(_) + | TypedElements::Date32(_) + | TypedElements::TimestampMicro(_) + ) + } + + /// Check if value at given index is null. + #[inline] + fn is_null_at(&self, idx: usize) -> bool { + match self { + TypedElements::Boolean(arr) => arr.is_null(idx), + TypedElements::Int8(arr) => arr.is_null(idx), + TypedElements::Int16(arr) => arr.is_null(idx), + TypedElements::Int32(arr) => arr.is_null(idx), + TypedElements::Int64(arr) => arr.is_null(idx), + TypedElements::Float32(arr) => arr.is_null(idx), + TypedElements::Float64(arr) => arr.is_null(idx), + TypedElements::Date32(arr) => arr.is_null(idx), + TypedElements::TimestampMicro(arr) => arr.is_null(idx), + TypedElements::Decimal128(arr, _) => arr.is_null(idx), + TypedElements::String(arr) => arr.is_null(idx), + TypedElements::LargeString(arr) => arr.is_null(idx), + TypedElements::Binary(arr) => arr.is_null(idx), + TypedElements::LargeBinary(arr) => arr.is_null(idx), + TypedElements::FixedSizeBinary(arr) => arr.is_null(idx), + TypedElements::Other(arr, _) => arr.is_null(idx), + } + } + + /// Check if this is a fixed-width type (value fits in 8-byte slot). + #[inline] + fn is_fixed_width(&self) -> bool { + match self { + TypedElements::Boolean(_) + | TypedElements::Int8(_) + | TypedElements::Int16(_) + | TypedElements::Int32(_) + | TypedElements::Int64(_) + | TypedElements::Float32(_) + | TypedElements::Float64(_) + | TypedElements::Date32(_) + | TypedElements::TimestampMicro(_) => true, + TypedElements::Decimal128(_, p) => *p <= MAX_LONG_DIGITS, + _ => false, + } + } + + /// Get fixed-width value as i64 for the 8-byte field slot. + #[inline] + fn get_fixed_value(&self, idx: usize) -> i64 { + match self { + TypedElements::Boolean(arr) => { + if arr.value(idx) { + 1 + } else { + 0 + } + } + TypedElements::Int8(arr) => arr.value(idx) as i64, + TypedElements::Int16(arr) => arr.value(idx) as i64, + TypedElements::Int32(arr) => arr.value(idx) as i64, + TypedElements::Int64(arr) => arr.value(idx), + TypedElements::Float32(arr) => (arr.value(idx).to_bits() as i32) as i64, + TypedElements::Float64(arr) => arr.value(idx).to_bits() as i64, + TypedElements::Date32(arr) => arr.value(idx) as i64, + TypedElements::TimestampMicro(arr) => arr.value(idx), + TypedElements::Decimal128(arr, _) => arr.value(idx) as i64, + _ => 0, // Should not be called for variable-length types + } + } + + /// Write variable-length data to buffer. Returns length written (0 for fixed-width). + fn write_variable_value( + &self, + buffer: &mut Vec, + idx: usize, + base_offset: usize, + ) -> CometResult { + match self { + TypedElements::String(arr) => { + let bytes = arr.value(idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedElements::LargeString(arr) => { + let bytes = arr.value(idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedElements::Binary(arr) => { + let bytes = arr.value(idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedElements::LargeBinary(arr) => { + let bytes = arr.value(idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedElements::FixedSizeBinary(arr) => { + let bytes = arr.value(idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedElements::Decimal128(arr, precision) if *precision > MAX_LONG_DIGITS => { + let bytes = i128_to_spark_decimal_bytes(arr.value(idx)); + let len = bytes.len(); + buffer.extend_from_slice(&bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + TypedElements::Other(arr, element_type) => { + write_nested_variable_to_buffer(buffer, element_type, arr, idx, base_offset) + } + _ => Ok(0), // Fixed-width types + } + } + + /// Write a range of elements to buffer in UnsafeArrayData format. + /// Returns the total bytes written (including header). + fn write_range_to_buffer( + &self, + buffer: &mut Vec, + start_idx: usize, + num_elements: usize, + ) -> CometResult { + let element_size = self.element_size(); + let array_start = buffer.len(); + let element_bitset_width = ColumnarToRowContext::calculate_bitset_width(num_elements); + + // Write number of elements + buffer.extend_from_slice(&(num_elements as i64).to_le_bytes()); + + // Reserve space for null bitset + let null_bitset_start = buffer.len(); + buffer.resize(null_bitset_start + element_bitset_width, 0); + + // Reserve space for element values + let elements_start = buffer.len(); + let elements_total_size = round_up_to_8(num_elements * element_size); + buffer.resize(elements_start + elements_total_size, 0); + + // Try bulk copy for primitive types + if self.supports_bulk_copy() { + self.bulk_copy_range( + buffer, + null_bitset_start, + elements_start, + start_idx, + num_elements, + ); + return Ok(buffer.len() - array_start); + } + + // Handle other types element by element + self.write_elements_slow( + buffer, + array_start, + null_bitset_start, + elements_start, + element_size, + start_idx, + num_elements, + ) + } + + /// Bulk copy primitive values from a range. + fn bulk_copy_range( + &self, + buffer: &mut [u8], + null_bitset_start: usize, + elements_start: usize, + start_idx: usize, + num_elements: usize, + ) { + macro_rules! bulk_copy_range { + ($arr:expr, $elem_size:expr) => {{ + let values_slice = $arr.values(); + let byte_len = num_elements * $elem_size; + let src_start = start_idx * $elem_size; + let src_bytes = unsafe { + std::slice::from_raw_parts( + (values_slice.as_ptr() as *const u8).add(src_start), + byte_len, + ) + }; + buffer[elements_start..elements_start + byte_len].copy_from_slice(src_bytes); + + // Set null bits + if $arr.null_count() > 0 { + for i in 0..num_elements { + if $arr.is_null(start_idx + i) { + let word_idx = i / 64; + let bit_idx = i % 64; + let word_offset = null_bitset_start + word_idx * 8; + let mut word = i64::from_le_bytes( + buffer[word_offset..word_offset + 8].try_into().unwrap(), + ); + word |= 1i64 << bit_idx; + buffer[word_offset..word_offset + 8] + .copy_from_slice(&word.to_le_bytes()); + } + } + } + }}; + } + + match self { + TypedElements::Int8(arr) => bulk_copy_range!(arr, 1), + TypedElements::Int16(arr) => bulk_copy_range!(arr, 2), + TypedElements::Int32(arr) => bulk_copy_range!(arr, 4), + TypedElements::Int64(arr) => bulk_copy_range!(arr, 8), + TypedElements::Float32(arr) => bulk_copy_range!(arr, 4), + TypedElements::Float64(arr) => bulk_copy_range!(arr, 8), + TypedElements::Date32(arr) => bulk_copy_range!(arr, 4), + TypedElements::TimestampMicro(arr) => bulk_copy_range!(arr, 8), + _ => {} // Should not reach here due to supports_bulk_copy check + } + } + + /// Slow path for non-bulk-copyable types. + #[allow(clippy::too_many_arguments)] + fn write_elements_slow( + &self, + buffer: &mut Vec, + array_start: usize, + null_bitset_start: usize, + elements_start: usize, + element_size: usize, + start_idx: usize, + num_elements: usize, + ) -> CometResult { + match self { + TypedElements::Boolean(arr) => { + for i in 0..num_elements { + let src_idx = start_idx + i; + if arr.is_null(src_idx) { + set_null_bit(buffer, null_bitset_start, i); + } else { + buffer[elements_start + i] = if arr.value(src_idx) { 1 } else { 0 }; + } + } + } + TypedElements::Decimal128(arr, precision) if *precision <= MAX_LONG_DIGITS => { + for i in 0..num_elements { + let src_idx = start_idx + i; + if arr.is_null(src_idx) { + set_null_bit(buffer, null_bitset_start, i); + } else { + let slot_offset = elements_start + i * 8; + let value = arr.value(src_idx) as i64; + buffer[slot_offset..slot_offset + 8].copy_from_slice(&value.to_le_bytes()); + } + } + } + TypedElements::Decimal128(arr, _) => { + // Large decimal - variable length + for i in 0..num_elements { + let src_idx = start_idx + i; + if arr.is_null(src_idx) { + set_null_bit(buffer, null_bitset_start, i); + } else { + let bytes = i128_to_spark_decimal_bytes(arr.value(src_idx)); + let len = bytes.len(); + buffer.extend_from_slice(&bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + + let data_offset = buffer.len() - round_up_to_8(len) - array_start; + let offset_and_len = ((data_offset as i64) << 32) | (len as i64); + let slot_offset = elements_start + i * 8; + buffer[slot_offset..slot_offset + 8] + .copy_from_slice(&offset_and_len.to_le_bytes()); + } + } + } + TypedElements::String(arr) => { + for i in 0..num_elements { + let src_idx = start_idx + i; + if arr.is_null(src_idx) { + set_null_bit(buffer, null_bitset_start, i); + } else { + let bytes = arr.value(src_idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + + let data_offset = buffer.len() - round_up_to_8(len) - array_start; + let offset_and_len = ((data_offset as i64) << 32) | (len as i64); + let slot_offset = elements_start + i * 8; + buffer[slot_offset..slot_offset + 8] + .copy_from_slice(&offset_and_len.to_le_bytes()); + } + } + } + TypedElements::LargeString(arr) => { + for i in 0..num_elements { + let src_idx = start_idx + i; + if arr.is_null(src_idx) { + set_null_bit(buffer, null_bitset_start, i); + } else { + let bytes = arr.value(src_idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + + let data_offset = buffer.len() - round_up_to_8(len) - array_start; + let offset_and_len = ((data_offset as i64) << 32) | (len as i64); + let slot_offset = elements_start + i * 8; + buffer[slot_offset..slot_offset + 8] + .copy_from_slice(&offset_and_len.to_le_bytes()); + } + } + } + TypedElements::Binary(arr) => { + for i in 0..num_elements { + let src_idx = start_idx + i; + if arr.is_null(src_idx) { + set_null_bit(buffer, null_bitset_start, i); + } else { + let bytes = arr.value(src_idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + + let data_offset = buffer.len() - round_up_to_8(len) - array_start; + let offset_and_len = ((data_offset as i64) << 32) | (len as i64); + let slot_offset = elements_start + i * 8; + buffer[slot_offset..slot_offset + 8] + .copy_from_slice(&offset_and_len.to_le_bytes()); + } + } + } + TypedElements::LargeBinary(arr) => { + for i in 0..num_elements { + let src_idx = start_idx + i; + if arr.is_null(src_idx) { + set_null_bit(buffer, null_bitset_start, i); + } else { + let bytes = arr.value(src_idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + + let data_offset = buffer.len() - round_up_to_8(len) - array_start; + let offset_and_len = ((data_offset as i64) << 32) | (len as i64); + let slot_offset = elements_start + i * 8; + buffer[slot_offset..slot_offset + 8] + .copy_from_slice(&offset_and_len.to_le_bytes()); + } + } + } + TypedElements::Other(arr, element_type) => { + // Fall back to old method for nested types + for i in 0..num_elements { + let src_idx = start_idx + i; + if arr.is_null(src_idx) { + set_null_bit(buffer, null_bitset_start, i); + } else { + let slot_offset = elements_start + i * element_size; + let var_len = write_nested_variable_to_buffer( + buffer, + element_type, + arr, + src_idx, + array_start, + )?; + + if var_len > 0 { + let padded_len = round_up_to_8(var_len); + let data_offset = buffer.len() - padded_len - array_start; + let offset_and_len = ((data_offset as i64) << 32) | (var_len as i64); + buffer[slot_offset..slot_offset + 8] + .copy_from_slice(&offset_and_len.to_le_bytes()); + } else { + let value = get_field_value(element_type, arr, src_idx)?; + write_array_element(buffer, element_type, value, slot_offset); + } + } + } + } + _ => { + // Should not reach here - all cases covered above + } + } + Ok(buffer.len() - array_start) + } +} + +/// Helper to set a null bit in the buffer. +#[inline] +fn set_null_bit(buffer: &mut [u8], null_bitset_start: usize, idx: usize) { + let word_idx = idx / 64; + let bit_idx = idx % 64; + let word_offset = null_bitset_start + word_idx * 8; + let mut word = i64::from_le_bytes(buffer[word_offset..word_offset + 8].try_into().unwrap()); + word |= 1i64 << bit_idx; + buffer[word_offset..word_offset + 8].copy_from_slice(&word.to_le_bytes()); +} + +/// Check if a data type is fixed-width for UnsafeRow purposes. +/// Fixed-width types are stored directly in the 8-byte field slot. +#[inline] +fn is_fixed_width(data_type: &DataType) -> bool { + match data_type { + DataType::Boolean + | DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Date32 + | DataType::Timestamp(TimeUnit::Microsecond, _) => true, + DataType::Decimal128(p, _) => *p <= MAX_LONG_DIGITS, + _ => false, + } +} + +/// Check if all columns in a schema are fixed-width. +#[inline] +fn is_all_fixed_width(schema: &[DataType]) -> bool { + schema.iter().all(is_fixed_width) +} + +/// Context for columnar to row conversion. +/// +/// This struct maintains the output buffer and schema information needed for +/// converting Arrow columnar data to Spark UnsafeRow format. The buffer is +/// reused across multiple `convert` calls to minimize allocations. +pub struct ColumnarToRowContext { + /// The Arrow data types for each column. + schema: Vec, + /// The output buffer containing converted rows. + /// Layout: [Row0][Row1]...[RowN] where each row is an UnsafeRow. + buffer: Vec, + /// Byte offset where each row starts in the buffer. + offsets: Vec, + /// Byte length of each row. + lengths: Vec, + /// Pre-calculated null bitset width in bytes. + null_bitset_width: usize, + /// Pre-calculated fixed-width portion size in bytes (null bitset + 8 bytes per field). + fixed_width_size: usize, + /// Maximum batch size for pre-allocation. + _batch_size: usize, + /// Whether all columns are fixed-width (enables fast path). + all_fixed_width: bool, +} + +impl ColumnarToRowContext { + /// Creates a new ColumnarToRowContext with the given schema. + /// + /// # Arguments + /// + /// * `schema` - The Arrow data types for each column. + /// * `batch_size` - Maximum number of rows expected per batch (for pre-allocation). + pub fn new(schema: Vec, batch_size: usize) -> Self { + let num_fields = schema.len(); + let null_bitset_width = Self::calculate_bitset_width(num_fields); + let fixed_width_size = null_bitset_width + num_fields * 8; + let all_fixed_width = is_all_fixed_width(&schema); + + // Pre-allocate buffer for maximum batch size + // For fixed-width schemas, we know exact size; otherwise estimate + let estimated_row_size = if all_fixed_width { + fixed_width_size + } else { + fixed_width_size + 64 // Conservative estimate for variable-length data + }; + let initial_capacity = batch_size * estimated_row_size; + + Self { + schema, + buffer: Vec::with_capacity(initial_capacity), + offsets: Vec::with_capacity(batch_size), + lengths: Vec::with_capacity(batch_size), + null_bitset_width, + fixed_width_size, + _batch_size: batch_size, + all_fixed_width, + } + } + + /// Calculate the width of the null bitset in bytes. + /// This matches Spark's `UnsafeRow.calculateBitSetWidthInBytes`. + #[inline] + pub const fn calculate_bitset_width(num_fields: usize) -> usize { + num_fields.div_ceil(64) * 8 + } + + /// Round up to the nearest multiple of 8 for alignment. + #[inline] + const fn round_up_to_8(value: usize) -> usize { + value.div_ceil(8) * 8 + } + + /// Converts Arrow arrays to Spark UnsafeRow format. + /// + /// # Arguments + /// + /// * `arrays` - The Arrow arrays to convert, one per column. + /// * `num_rows` - The number of rows to convert. + /// + /// # Returns + /// + /// A tuple containing: + /// - A pointer to the output buffer + /// - A reference to the offsets array + /// - A reference to the lengths array + pub fn convert( + &mut self, + arrays: &[ArrayRef], + num_rows: usize, + ) -> CometResult<(*const u8, &[i32], &[i32])> { + if arrays.len() != self.schema.len() { + return Err(CometError::Internal(format!( + "Column count mismatch: expected {}, got {}", + self.schema.len(), + arrays.len() + ))); + } + + // Unpack any dictionary arrays to their underlying value type + // This is needed because Parquet may return dictionary-encoded arrays + // even when the schema expects a specific type like Decimal128 + let arrays: Vec = arrays + .iter() + .zip(self.schema.iter()) + .map(|(arr, schema_type)| Self::maybe_cast_to_schema_type(arr, schema_type)) + .collect::>>()?; + let arrays = arrays.as_slice(); + + // Clear previous data + self.buffer.clear(); + self.offsets.clear(); + self.lengths.clear(); + + // Reserve space for offsets and lengths + self.offsets.reserve(num_rows); + self.lengths.reserve(num_rows); + + // Use fast path for fixed-width-only schemas + if self.all_fixed_width { + return self.convert_fixed_width(arrays, num_rows); + } + + // Pre-downcast all arrays to avoid type dispatch in inner loop + let typed_arrays: Vec = arrays + .iter() + .zip(self.schema.iter()) + .map(|(arr, dt)| TypedArray::from_array(arr, dt)) + .collect::>>()?; + + // Pre-compute variable-length column indices (once per batch, not per row) + let var_len_indices: Vec = typed_arrays + .iter() + .enumerate() + .filter(|(_, arr)| arr.is_variable_length()) + .map(|(idx, _)| idx) + .collect(); + + // Process each row (general path for variable-length data) + for row_idx in 0..num_rows { + let row_start = self.buffer.len(); + self.offsets.push(row_start as i32); + + // Write fixed-width portion (null bitset + field values) + self.write_row_typed(&typed_arrays, &var_len_indices, row_idx)?; + + let row_end = self.buffer.len(); + self.lengths.push((row_end - row_start) as i32); + } + + Ok((self.buffer.as_ptr(), &self.offsets, &self.lengths)) + } + + /// Casts an array to match the expected schema type if needed. + /// This handles cases where: + /// 1. Parquet returns dictionary-encoded arrays but the schema expects a non-dictionary type + /// 2. Parquet returns NullArray when all values are null, but the schema expects a typed array + /// 3. Parquet returns Int32/Int64 for small-precision decimals but schema expects Decimal128 + fn maybe_cast_to_schema_type( + array: &ArrayRef, + schema_type: &DataType, + ) -> CometResult { + let actual_type = array.data_type(); + + // If types already match, no cast needed + if actual_type == schema_type { + return Ok(Arc::clone(array)); + } + + match (actual_type, schema_type) { + (DataType::Dictionary(_, _), schema) + if !matches!(schema, DataType::Dictionary(_, _)) => + { + // Unpack dictionary if the schema type is not a dictionary + let options = CastOptions::default(); + cast_with_options(array, schema_type, &options).map_err(|e| { + CometError::Internal(format!( + "Failed to unpack dictionary array from {:?} to {:?}: {}", + actual_type, schema_type, e + )) + }) + } + (DataType::Null, _) => { + // Cast NullArray to the expected schema type + // This happens when all values in a column are null + let options = CastOptions::default(); + cast_with_options(array, schema_type, &options).map_err(|e| { + CometError::Internal(format!( + "Failed to cast NullArray to {:?}: {}", + schema_type, e + )) + }) + } + (DataType::Int32, DataType::Decimal128(precision, scale)) => { + // Parquet stores small-precision decimals as Int32 for efficiency. + // When COMET_USE_DECIMAL_128 is false, BatchReader produces these types. + // The Int32 value is already scaled (e.g., -1 means -0.01 for scale 2). + // We need to reinterpret (not cast) to Decimal128 preserving the value. + let int_array = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal("Failed to downcast to Int32Array".to_string()) + })?; + let decimal_array: Decimal128Array = int_array + .iter() + .map(|v| v.map(|x| x as i128)) + .collect::() + .with_precision_and_scale(*precision, *scale) + .map_err(|e| { + CometError::Internal(format!("Invalid decimal precision/scale: {}", e)) + })?; + Ok(Arc::new(decimal_array)) + } + (DataType::Int64, DataType::Decimal128(precision, scale)) => { + // Same as Int32 but for medium-precision decimals stored as Int64. + let int_array = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal("Failed to downcast to Int64Array".to_string()) + })?; + let decimal_array: Decimal128Array = int_array + .iter() + .map(|v| v.map(|x| x as i128)) + .collect::() + .with_precision_and_scale(*precision, *scale) + .map_err(|e| { + CometError::Internal(format!("Invalid decimal precision/scale: {}", e)) + })?; + Ok(Arc::new(decimal_array)) + } + _ => Ok(Arc::clone(array)), + } + } + + /// Fast path for schemas with only fixed-width columns. + /// Pre-allocates entire buffer and processes more efficiently. + fn convert_fixed_width( + &mut self, + arrays: &[ArrayRef], + num_rows: usize, + ) -> CometResult<(*const u8, &[i32], &[i32])> { + let row_size = self.fixed_width_size; + let total_size = row_size * num_rows; + let null_bitset_width = self.null_bitset_width; + + // Pre-allocate entire buffer at once (all zeros) + self.buffer.resize(total_size, 0); + + // Pre-fill offsets and lengths (constant for fixed-width) + let row_size_i32 = row_size as i32; + for row_idx in 0..num_rows { + self.offsets.push((row_idx * row_size) as i32); + self.lengths.push(row_size_i32); + } + + // Process column by column for better cache locality + for (col_idx, array) in arrays.iter().enumerate() { + let field_offset_in_row = null_bitset_width + col_idx * 8; + + // Write values for all rows in this column + self.write_column_fixed_width( + array, + &self.schema[col_idx].clone(), + col_idx, + field_offset_in_row, + row_size, + num_rows, + )?; + } + + Ok((self.buffer.as_ptr(), &self.offsets, &self.lengths)) + } + + /// Write a fixed-width column's values for all rows. + /// Processes column-by-column for better cache locality. + fn write_column_fixed_width( + &mut self, + array: &ArrayRef, + data_type: &DataType, + col_idx: usize, + field_offset_in_row: usize, + row_size: usize, + num_rows: usize, + ) -> CometResult<()> { + // Handle nulls first - set null bits + if array.null_count() > 0 { + let word_idx = col_idx / 64; + let bit_idx = col_idx % 64; + let bit_mask = 1i64 << bit_idx; + + for row_idx in 0..num_rows { + if array.is_null(row_idx) { + let row_start = row_idx * row_size; + let word_offset = row_start + word_idx * 8; + let mut word = i64::from_le_bytes( + self.buffer[word_offset..word_offset + 8] + .try_into() + .unwrap(), + ); + word |= bit_mask; + self.buffer[word_offset..word_offset + 8].copy_from_slice(&word.to_le_bytes()); + } + } + } + + // Write non-null values using type-specific fast paths + match data_type { + DataType::Boolean => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to BooleanArray, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset] = if arr.value(row_idx) { 1 } else { 0 }; + } + } + } + DataType::Int8 => { + let arr = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int8Array, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset..offset + 8] + .copy_from_slice(&(arr.value(row_idx) as i64).to_le_bytes()); + } + } + } + DataType::Int16 => { + let arr = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int16Array, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset..offset + 8] + .copy_from_slice(&(arr.value(row_idx) as i64).to_le_bytes()); + } + } + } + DataType::Int32 => { + let arr = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int32Array, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset..offset + 8] + .copy_from_slice(&(arr.value(row_idx) as i64).to_le_bytes()); + } + } + } + DataType::Int64 => { + let arr = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int64Array, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset..offset + 8] + .copy_from_slice(&arr.value(row_idx).to_le_bytes()); + } + } + } + DataType::Float32 => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Float32Array, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset..offset + 8] + .copy_from_slice(&(arr.value(row_idx).to_bits() as i64).to_le_bytes()); + } + } + } + DataType::Float64 => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Float64Array, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset..offset + 8] + .copy_from_slice(&(arr.value(row_idx).to_bits() as i64).to_le_bytes()); + } + } + } + DataType::Date32 => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Date32Array, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset..offset + 8] + .copy_from_slice(&(arr.value(row_idx) as i64).to_le_bytes()); + } + } + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to TimestampMicrosecondArray, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset..offset + 8] + .copy_from_slice(&arr.value(row_idx).to_le_bytes()); + } + } + } + DataType::Decimal128(precision, _) if *precision <= MAX_LONG_DIGITS => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Decimal128Array, actual type: {:?}", + array.data_type() + )) + })?; + for row_idx in 0..num_rows { + if !arr.is_null(row_idx) { + let offset = row_idx * row_size + field_offset_in_row; + self.buffer[offset..offset + 8] + .copy_from_slice(&(arr.value(row_idx) as i64).to_le_bytes()); + } + } + } + _ => { + return Err(CometError::Internal(format!( + "Unexpected non-fixed-width type in fast path: {:?}", + data_type + ))); + } + } + + Ok(()) + } + + /// Writes a complete row using pre-downcast TypedArrays. + /// This avoids type dispatch overhead in the inner loop. + fn write_row_typed( + &mut self, + typed_arrays: &[TypedArray], + var_len_indices: &[usize], + row_idx: usize, + ) -> CometResult<()> { + let row_start = self.buffer.len(); + let null_bitset_width = self.null_bitset_width; + let fixed_width_size = self.fixed_width_size; + + // Extend buffer for fixed-width portion + self.buffer.resize(row_start + fixed_width_size, 0); + + // First pass: write null bits and fixed-width values + for (col_idx, typed_arr) in typed_arrays.iter().enumerate() { + let is_null = typed_arr.is_null(row_idx); + + if is_null { + // Set null bit + let word_idx = col_idx / 64; + let bit_idx = col_idx % 64; + let word_offset = row_start + word_idx * 8; + + let mut word = i64::from_le_bytes( + self.buffer[word_offset..word_offset + 8] + .try_into() + .unwrap(), + ); + word |= 1i64 << bit_idx; + self.buffer[word_offset..word_offset + 8].copy_from_slice(&word.to_le_bytes()); + } else if !typed_arr.is_variable_length() { + // Write fixed-width field value (skip variable-length, they're handled in pass 2) + let field_offset = row_start + null_bitset_width + col_idx * 8; + let value = typed_arr.get_fixed_value(row_idx); + self.buffer[field_offset..field_offset + 8].copy_from_slice(&value.to_le_bytes()); + } + } + + // Second pass: write variable-length data (only iterate over var-len columns) + for &col_idx in var_len_indices { + let typed_arr = &typed_arrays[col_idx]; + if typed_arr.is_null(row_idx) { + continue; + } + + // Write variable-length data directly to buffer + let actual_len = typed_arr.write_variable_to_buffer(&mut self.buffer, row_idx)?; + if actual_len > 0 { + // Calculate offset: buffer grew by padded_len, but we need offset to start of data + let padded_len = Self::round_up_to_8(actual_len); + let current_offset = self.buffer.len() - padded_len - row_start; + + // Update the field slot with (offset << 32) | length + let offset_and_len = ((current_offset as i64) << 32) | (actual_len as i64); + let field_offset = row_start + null_bitset_width + col_idx * 8; + self.buffer[field_offset..field_offset + 8] + .copy_from_slice(&offset_and_len.to_le_bytes()); + } + } + + Ok(()) + } + + /// Returns a pointer to the buffer. + pub fn buffer_ptr(&self) -> *const u8 { + self.buffer.as_ptr() + } + + /// Returns the schema. + pub fn schema(&self) -> &[DataType] { + &self.schema + } +} + +/// Gets the fixed-width value for a field as i64. +#[inline] +fn get_field_value(data_type: &DataType, array: &ArrayRef, row_idx: usize) -> CometResult { + // Use the actual array type for dispatching to handle type mismatches + let actual_type = array.data_type(); + + match actual_type { + DataType::Boolean => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to BooleanArray for type {:?}", + actual_type + )) + })?; + Ok(if arr.value(row_idx) { 1i64 } else { 0i64 }) + } + DataType::Int8 => { + let arr = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int8Array for type {:?}", + actual_type + )) + })?; + Ok(arr.value(row_idx) as i64) + } + DataType::Int16 => { + let arr = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int16Array for type {:?}", + actual_type + )) + })?; + Ok(arr.value(row_idx) as i64) + } + DataType::Int32 => { + let arr = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int32Array for type {:?}", + actual_type + )) + })?; + Ok(arr.value(row_idx) as i64) + } + DataType::Int64 => { + let arr = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Int64Array for type {:?}", + actual_type + )) + })?; + Ok(arr.value(row_idx)) + } + DataType::Float32 => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Float32Array for type {:?}", + actual_type + )) + })?; + Ok(arr.value(row_idx).to_bits() as i64) + } + DataType::Float64 => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Float64Array for type {:?}", + actual_type + )) + })?; + Ok(arr.value(row_idx).to_bits() as i64) + } + DataType::Date32 => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Date32Array for type {:?}", + actual_type + )) + })?; + Ok(arr.value(row_idx) as i64) + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to TimestampMicrosecondArray for type {:?}", + actual_type + )) + })?; + Ok(arr.value(row_idx)) + } + DataType::Decimal128(precision, _) if *precision <= MAX_LONG_DIGITS => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Decimal128Array for type {:?}", + actual_type + )) + })?; + Ok(arr.value(row_idx) as i64) + } + // Variable-length types use placeholder (will be overwritten by get_variable_length_data) + DataType::Utf8 + | DataType::LargeUtf8 + | DataType::Binary + | DataType::LargeBinary + | DataType::Decimal128(_, _) + | DataType::Struct(_) + | DataType::List(_) + | DataType::LargeList(_) + | DataType::Map(_, _) + | DataType::Dictionary(_, _) => Ok(0i64), + _ => { + // Check if the schema type is a known type that we should handle + match data_type { + DataType::Boolean + | DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Date32 + | DataType::Timestamp(TimeUnit::Microsecond, _) + | DataType::Decimal128(_, _) => Err(CometError::Internal(format!( + "Type mismatch in get_field_value: schema expects {:?} but actual array type is {:?}", + data_type, actual_type + ))), + // If schema is also a variable-length type, return placeholder + DataType::Utf8 + | DataType::LargeUtf8 + | DataType::Binary + | DataType::LargeBinary + | DataType::Struct(_) + | DataType::List(_) + | DataType::LargeList(_) + | DataType::Map(_, _) + | DataType::Dictionary(_, _) => Ok(0i64), + _ => Err(CometError::Internal(format!( + "Unsupported data type for columnar to row conversion: schema={:?}, actual={:?}", + data_type, actual_type + ))), + } + } + } +} + +/// Writes dictionary-encoded value directly to buffer. +#[inline] +fn write_dictionary_to_buffer( + buffer: &mut Vec, + array: &ArrayRef, + row_idx: usize, + key_type: &DataType, + value_type: &DataType, +) -> CometResult { + match key_type { + DataType::Int8 => { + write_dictionary_to_buffer_with_key::(buffer, array, row_idx, value_type) + } + DataType::Int16 => { + write_dictionary_to_buffer_with_key::(buffer, array, row_idx, value_type) + } + DataType::Int32 => { + write_dictionary_to_buffer_with_key::(buffer, array, row_idx, value_type) + } + DataType::Int64 => { + write_dictionary_to_buffer_with_key::(buffer, array, row_idx, value_type) + } + DataType::UInt8 => { + write_dictionary_to_buffer_with_key::(buffer, array, row_idx, value_type) + } + DataType::UInt16 => { + write_dictionary_to_buffer_with_key::(buffer, array, row_idx, value_type) + } + DataType::UInt32 => { + write_dictionary_to_buffer_with_key::(buffer, array, row_idx, value_type) + } + DataType::UInt64 => { + write_dictionary_to_buffer_with_key::(buffer, array, row_idx, value_type) + } + _ => Err(CometError::Internal(format!( + "Unsupported dictionary key type: {:?}", + key_type + ))), + } +} + +/// Writes dictionary value directly to buffer with specific key type. +#[inline] +fn write_dictionary_to_buffer_with_key( + buffer: &mut Vec, + array: &ArrayRef, + row_idx: usize, + value_type: &DataType, +) -> CometResult { + let dict_array = array + .as_any() + .downcast_ref::>() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to DictionaryArray<{:?}>", + std::any::type_name::() + )) + })?; + + let values = dict_array.values(); + let key_idx = dict_array.keys().value(row_idx).to_usize().ok_or_else(|| { + CometError::Internal("Dictionary key index out of usize range".to_string()) + })?; + + match value_type { + DataType::Utf8 => { + let string_values = values + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast dictionary values to StringArray, actual type: {:?}", + values.data_type() + )) + })?; + let bytes = string_values.value(key_idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + DataType::LargeUtf8 => { + let string_values = values + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast dictionary values to LargeStringArray, actual type: {:?}", + values.data_type() + )) + })?; + let bytes = string_values.value(key_idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + DataType::Binary => { + let binary_values = values + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast dictionary values to BinaryArray, actual type: {:?}", + values.data_type() + )) + })?; + let bytes = binary_values.value(key_idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + DataType::LargeBinary => { + let binary_values = values + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast dictionary values to LargeBinaryArray, actual type: {:?}", + values.data_type() + )) + })?; + let bytes = binary_values.value(key_idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + _ => Err(CometError::Internal(format!( + "Unsupported dictionary value type for direct buffer write: {:?}", + value_type + ))), + } +} + +/// Converts i128 to Spark's big-endian decimal byte format. +fn i128_to_spark_decimal_bytes(value: i128) -> Vec { + // Spark uses big-endian format for large decimals + let bytes = value.to_be_bytes(); + + // Find the minimum number of bytes needed (excluding leading sign-extension bytes) + let is_negative = value < 0; + let sign_byte = if is_negative { 0xFF } else { 0x00 }; + + let mut start = 0; + while start < 15 && bytes[start] == sign_byte { + // Check if the next byte's sign bit matches + let next_byte = bytes[start + 1]; + let has_correct_sign = if is_negative { + (next_byte & 0x80) != 0 + } else { + (next_byte & 0x80) == 0 + }; + if has_correct_sign { + start += 1; + } else { + break; + } + } + + bytes[start..].to_vec() +} + +/// Round up to the nearest multiple of 8 for alignment. +#[inline] +const fn round_up_to_8(value: usize) -> usize { + value.div_ceil(8) * 8 +} + +/// Writes a primitive value with the correct size for UnsafeArrayData. +#[inline] +fn write_array_element(buffer: &mut [u8], data_type: &DataType, value: i64, offset: usize) { + match data_type { + DataType::Boolean => { + buffer[offset] = if value != 0 { 1 } else { 0 }; + } + DataType::Int8 => { + buffer[offset] = value as u8; + } + DataType::Int16 => { + buffer[offset..offset + 2].copy_from_slice(&(value as i16).to_le_bytes()); + } + DataType::Int32 | DataType::Date32 => { + buffer[offset..offset + 4].copy_from_slice(&(value as i32).to_le_bytes()); + } + DataType::Float32 => { + buffer[offset..offset + 4].copy_from_slice(&(value as u32).to_le_bytes()); + } + // All 8-byte types + _ => { + buffer[offset..offset + 8].copy_from_slice(&value.to_le_bytes()); + } + } +} + +// ============================================================================= +// Optimized direct-write functions for complex types +// These write directly to the output buffer to avoid intermediate allocations. +// ============================================================================= + +/// Writes a struct value directly to the buffer using pre-downcast typed fields. +/// Returns the unpadded length written. +/// +/// This version uses pre-downcast TypedElements for each field, eliminating +/// per-row type dispatch overhead. +#[inline] +fn write_struct_to_buffer_typed( + buffer: &mut Vec, + _struct_array: &StructArray, + row_idx: usize, + _fields: &arrow::datatypes::Fields, + typed_fields: &[TypedElements], +) -> CometResult { + let num_fields = typed_fields.len(); + let nested_bitset_width = ColumnarToRowContext::calculate_bitset_width(num_fields); + let nested_fixed_size = nested_bitset_width + num_fields * 8; + + // Remember where this struct starts in the buffer + let struct_start = buffer.len(); + + // Reserve space for fixed-width portion (zeros for null bits and field slots) + buffer.resize(struct_start + nested_fixed_size, 0); + + // Write each field using pre-downcast types + for (field_idx, typed_field) in typed_fields.iter().enumerate() { + if typed_field.is_null_at(row_idx) { + // Set null bit in nested struct + set_null_bit(buffer, struct_start, field_idx); + } else { + let field_offset = struct_start + nested_bitset_width + field_idx * 8; + + if typed_field.is_fixed_width() { + // Fixed-width field - use pre-downcast accessor + let value = typed_field.get_fixed_value(row_idx); + buffer[field_offset..field_offset + 8].copy_from_slice(&value.to_le_bytes()); + } else { + // Variable-length field - use pre-downcast writer + let var_len = typed_field.write_variable_value(buffer, row_idx, struct_start)?; + if var_len > 0 { + let padded_len = round_up_to_8(var_len); + let data_offset = buffer.len() - padded_len - struct_start; + let offset_and_len = ((data_offset as i64) << 32) | (var_len as i64); + buffer[field_offset..field_offset + 8] + .copy_from_slice(&offset_and_len.to_le_bytes()); + } + } + } + } + + Ok(buffer.len() - struct_start) +} + +/// Writes a struct value directly to the buffer. +/// Returns the unpadded length written. +/// +/// Processes each field using inline type dispatch to avoid allocation overhead. +/// This is used for nested structs where we don't have pre-downcast fields. +fn write_struct_to_buffer( + buffer: &mut Vec, + struct_array: &StructArray, + row_idx: usize, + fields: &arrow::datatypes::Fields, +) -> CometResult { + let num_fields = fields.len(); + let nested_bitset_width = ColumnarToRowContext::calculate_bitset_width(num_fields); + let nested_fixed_size = nested_bitset_width + num_fields * 8; + + // Remember where this struct starts in the buffer + let struct_start = buffer.len(); + + // Reserve space for fixed-width portion (zeros for null bits and field slots) + buffer.resize(struct_start + nested_fixed_size, 0); + + // Write each field with inline type handling (no allocation) + for (field_idx, field) in fields.iter().enumerate() { + let column = struct_array.column(field_idx); + let data_type = field.data_type(); + + if column.is_null(row_idx) { + // Set null bit in nested struct + set_null_bit(buffer, struct_start, field_idx); + } else { + let field_offset = struct_start + nested_bitset_width + field_idx * 8; + + // Inline type dispatch for fixed-width types (most common case) + let value = match data_type { + DataType::Boolean => { + let arr = column.as_any().downcast_ref::().unwrap(); + Some(if arr.value(row_idx) { 1i64 } else { 0i64 }) + } + DataType::Int8 => { + let arr = column.as_any().downcast_ref::().unwrap(); + Some(arr.value(row_idx) as i64) + } + DataType::Int16 => { + let arr = column.as_any().downcast_ref::().unwrap(); + Some(arr.value(row_idx) as i64) + } + DataType::Int32 => { + let arr = column.as_any().downcast_ref::().unwrap(); + Some(arr.value(row_idx) as i64) + } + DataType::Int64 => { + let arr = column.as_any().downcast_ref::().unwrap(); + Some(arr.value(row_idx)) + } + DataType::Float32 => { + let arr = column.as_any().downcast_ref::().unwrap(); + Some((arr.value(row_idx).to_bits() as i32) as i64) + } + DataType::Float64 => { + let arr = column.as_any().downcast_ref::().unwrap(); + Some(arr.value(row_idx).to_bits() as i64) + } + DataType::Date32 => { + let arr = column.as_any().downcast_ref::().unwrap(); + Some(arr.value(row_idx) as i64) + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + let arr = column + .as_any() + .downcast_ref::() + .unwrap(); + Some(arr.value(row_idx)) + } + DataType::Decimal128(p, _) if *p <= MAX_LONG_DIGITS => { + let arr = column.as_any().downcast_ref::().unwrap(); + Some(arr.value(row_idx) as i64) + } + _ => None, // Variable-length type + }; + + if let Some(v) = value { + // Fixed-width field + buffer[field_offset..field_offset + 8].copy_from_slice(&v.to_le_bytes()); + } else { + // Variable-length field + let var_len = write_nested_variable_to_buffer( + buffer, + data_type, + column, + row_idx, + struct_start, + )?; + if var_len > 0 { + let padded_len = round_up_to_8(var_len); + let data_offset = buffer.len() - padded_len - struct_start; + let offset_and_len = ((data_offset as i64) << 32) | (var_len as i64); + buffer[field_offset..field_offset + 8] + .copy_from_slice(&offset_and_len.to_le_bytes()); + } + } + } + } + + Ok(buffer.len() - struct_start) +} + +/// Writes a list value directly to the buffer in UnsafeArrayData format. +/// Returns the unpadded length written. +/// +/// This uses offsets directly to avoid per-row ArrayRef allocation. +#[inline] +fn write_list_to_buffer( + buffer: &mut Vec, + list_array: &ListArray, + row_idx: usize, + element_field: &arrow::datatypes::FieldRef, +) -> CometResult { + // Get offsets directly to avoid creating a sliced ArrayRef + let offsets = list_array.value_offsets(); + let start_offset = offsets[row_idx] as usize; + let end_offset = offsets[row_idx + 1] as usize; + let num_elements = end_offset - start_offset; + + // Pre-downcast the element array once + let element_array = list_array.values(); + let element_type = element_field.data_type(); + let typed_elements = TypedElements::from_array(element_array, element_type); + + // Write the range of elements + typed_elements.write_range_to_buffer(buffer, start_offset, num_elements) +} + +/// Writes a large list value directly to the buffer in UnsafeArrayData format. +/// Returns the unpadded length written. +/// +/// This uses offsets directly to avoid per-row ArrayRef allocation. +#[inline] +fn write_large_list_to_buffer( + buffer: &mut Vec, + list_array: &LargeListArray, + row_idx: usize, + element_field: &arrow::datatypes::FieldRef, +) -> CometResult { + // Get offsets directly to avoid creating a sliced ArrayRef + let offsets = list_array.value_offsets(); + let start_offset = offsets[row_idx] as usize; + let end_offset = offsets[row_idx + 1] as usize; + let num_elements = end_offset - start_offset; + + // Pre-downcast the element array once + let element_array = list_array.values(); + let element_type = element_field.data_type(); + let typed_elements = TypedElements::from_array(element_array, element_type); + + // Write the range of elements + typed_elements.write_range_to_buffer(buffer, start_offset, num_elements) +} + +/// Writes a map value directly to the buffer in UnsafeMapData format. +/// Returns the unpadded length written. +/// +/// This uses offsets directly to avoid per-row ArrayRef allocation. +fn write_map_to_buffer( + buffer: &mut Vec, + map_array: &MapArray, + row_idx: usize, + entries_field: &arrow::datatypes::FieldRef, +) -> CometResult { + // UnsafeMapData format: + // [key array size: 8 bytes][key array data][value array data] + let map_start = buffer.len(); + + // Get offsets directly to avoid creating a sliced ArrayRef + let offsets = map_array.value_offsets(); + let start_offset = offsets[row_idx] as usize; + let end_offset = offsets[row_idx + 1] as usize; + let num_entries = end_offset - start_offset; + + // Get keys and values from the underlying entries struct + let entries_array = map_array.entries(); + let keys = entries_array.column(0); + let values = entries_array.column(1); + + let (key_type, value_type) = if let DataType::Struct(fields) = entries_field.data_type() { + (fields[0].data_type().clone(), fields[1].data_type().clone()) + } else { + return Err(CometError::Internal(format!( + "Map entries field is not a struct: {:?}", + entries_field.data_type() + ))); + }; + + // Pre-downcast keys and values once + let typed_keys = TypedElements::from_array(keys, &key_type); + let typed_values = TypedElements::from_array(values, &value_type); + + // Placeholder for key array size + let key_size_offset = buffer.len(); + buffer.extend_from_slice(&0i64.to_le_bytes()); + + // Write key array using range + let key_array_start = buffer.len(); + typed_keys.write_range_to_buffer(buffer, start_offset, num_entries)?; + let key_array_size = (buffer.len() - key_array_start) as i64; + buffer[key_size_offset..key_size_offset + 8].copy_from_slice(&key_array_size.to_le_bytes()); + + // Write value array using range + typed_values.write_range_to_buffer(buffer, start_offset, num_entries)?; + + Ok(buffer.len() - map_start) +} + +/// Writes variable-length data for a nested field directly to buffer. +/// Used by struct, list, and map writers for their nested elements. +/// Returns the unpadded length written (0 if not variable-length). +fn write_nested_variable_to_buffer( + buffer: &mut Vec, + data_type: &DataType, + array: &ArrayRef, + row_idx: usize, + _base_offset: usize, +) -> CometResult { + let actual_type = array.data_type(); + + match actual_type { + DataType::Utf8 => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to StringArray for type {:?}", + actual_type + )) + })?; + let bytes = arr.value(row_idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + DataType::LargeUtf8 => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to LargeStringArray for type {:?}", + actual_type + )) + })?; + let bytes = arr.value(row_idx).as_bytes(); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + DataType::Binary => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to BinaryArray for type {:?}", + actual_type + )) + })?; + let bytes = arr.value(row_idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + DataType::LargeBinary => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to LargeBinaryArray for type {:?}", + actual_type + )) + })?; + let bytes = arr.value(row_idx); + let len = bytes.len(); + buffer.extend_from_slice(bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + DataType::Decimal128(precision, _) if *precision > MAX_LONG_DIGITS => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to Decimal128Array for type {:?}", + actual_type + )) + })?; + let bytes = i128_to_spark_decimal_bytes(arr.value(row_idx)); + let len = bytes.len(); + buffer.extend_from_slice(&bytes); + let padding = round_up_to_8(len) - len; + buffer.extend(std::iter::repeat_n(0u8, padding)); + Ok(len) + } + DataType::Struct(fields) => { + let struct_array = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to StructArray for type {:?}", + actual_type + )) + })?; + write_struct_to_buffer(buffer, struct_array, row_idx, fields) + } + DataType::List(field) => { + let list_array = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to ListArray for type {:?}", + actual_type + )) + })?; + write_list_to_buffer(buffer, list_array, row_idx, field) + } + DataType::LargeList(field) => { + let list_array = array + .as_any() + .downcast_ref::() + .ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to LargeListArray for type {:?}", + actual_type + )) + })?; + write_large_list_to_buffer(buffer, list_array, row_idx, field) + } + DataType::Map(field, _) => { + let map_array = array.as_any().downcast_ref::().ok_or_else(|| { + CometError::Internal(format!( + "Failed to downcast to MapArray for type {:?}", + actual_type + )) + })?; + write_map_to_buffer(buffer, map_array, row_idx, field) + } + DataType::Dictionary(key_type, value_type) => { + write_dictionary_to_buffer(buffer, array, row_idx, key_type, value_type) + } + // Check if schema type expects variable-length but actual type doesn't match + _ => match data_type { + DataType::Utf8 + | DataType::LargeUtf8 + | DataType::Binary + | DataType::LargeBinary + | DataType::Struct(_) + | DataType::List(_) + | DataType::LargeList(_) + | DataType::Map(_, _) => Err(CometError::Internal(format!( + "Type mismatch in nested write: schema expects {:?} but actual array type is {:?}", + data_type, actual_type + ))), + DataType::Decimal128(precision, _) if *precision > MAX_LONG_DIGITS => { + Err(CometError::Internal(format!( + "Type mismatch for large decimal: schema expects {:?} but actual is {:?}", + data_type, actual_type + ))) + } + _ => Ok(0), // Not a variable-length type + }, + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Arc; + + #[test] + fn test_bitset_width_calculation() { + assert_eq!(ColumnarToRowContext::calculate_bitset_width(0), 0); + assert_eq!(ColumnarToRowContext::calculate_bitset_width(1), 8); + assert_eq!(ColumnarToRowContext::calculate_bitset_width(64), 8); + assert_eq!(ColumnarToRowContext::calculate_bitset_width(65), 16); + assert_eq!(ColumnarToRowContext::calculate_bitset_width(128), 16); + assert_eq!(ColumnarToRowContext::calculate_bitset_width(129), 24); + } + + #[test] + fn test_round_up_to_8() { + assert_eq!(ColumnarToRowContext::round_up_to_8(0), 0); + assert_eq!(ColumnarToRowContext::round_up_to_8(1), 8); + assert_eq!(ColumnarToRowContext::round_up_to_8(7), 8); + assert_eq!(ColumnarToRowContext::round_up_to_8(8), 8); + assert_eq!(ColumnarToRowContext::round_up_to_8(9), 16); + } + + #[test] + fn test_convert_int_array() { + let schema = vec![DataType::Int32]; + let mut ctx = ColumnarToRowContext::new(schema, 100); + + let array: ArrayRef = Arc::new(Int32Array::from(vec![Some(1), Some(2), None, Some(4)])); + let arrays = vec![array]; + + let (ptr, offsets, lengths) = ctx.convert(&arrays, 4).unwrap(); + + assert!(!ptr.is_null()); + assert_eq!(offsets.len(), 4); + assert_eq!(lengths.len(), 4); + + // Each row should have: 8 bytes null bitset + 8 bytes for one field = 16 bytes + for len in lengths { + assert_eq!(*len, 16); + } + } + + #[test] + fn test_convert_multiple_columns() { + let schema = vec![DataType::Int32, DataType::Int64, DataType::Float64]; + let mut ctx = ColumnarToRowContext::new(schema, 100); + + let array1: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); + let array2: ArrayRef = Arc::new(Int64Array::from(vec![100i64, 200, 300])); + let array3: ArrayRef = Arc::new(Float64Array::from(vec![1.1, 2.2, 3.3])); + let arrays = vec![array1, array2, array3]; + + let (ptr, offsets, lengths) = ctx.convert(&arrays, 3).unwrap(); + + assert!(!ptr.is_null()); + assert_eq!(offsets.len(), 3); + assert_eq!(lengths.len(), 3); + + // Each row should have: 8 bytes null bitset + 24 bytes for three fields = 32 bytes + for len in lengths { + assert_eq!(*len, 32); + } + } + + #[test] + fn test_fixed_width_fast_path() { + // Test that the fixed-width fast path produces correct results + let schema = vec![DataType::Int32, DataType::Int64, DataType::Float64]; + let mut ctx = ColumnarToRowContext::new(schema.clone(), 100); + + // Verify that the context detects this as all fixed-width + assert!( + ctx.all_fixed_width, + "Schema should be detected as all fixed-width" + ); + + let array1: ArrayRef = Arc::new(Int32Array::from(vec![Some(1), None, Some(3)])); + let array2: ArrayRef = Arc::new(Int64Array::from(vec![Some(100i64), Some(200), None])); + let array3: ArrayRef = Arc::new(Float64Array::from(vec![1.5, 2.5, 3.5])); + let arrays = vec![array1, array2, array3]; + + let (ptr, offsets, lengths) = ctx.convert(&arrays, 3).unwrap(); + + assert!(!ptr.is_null()); + assert_eq!(offsets.len(), 3); + assert_eq!(lengths.len(), 3); + + // Each row: 8 bytes null bitset + 24 bytes for three fields = 32 bytes + let row_size = 32; + for (i, len) in lengths.iter().enumerate() { + assert_eq!( + *len, row_size as i32, + "Row {} should be {} bytes", + i, row_size + ); + } + + // Verify the actual data + let buffer = unsafe { std::slice::from_raw_parts(ptr, row_size * 3) }; + + // Row 0: int32=1 (not null), int64=100 (not null), float64=1.5 (not null) + let null_bitset_0 = i64::from_le_bytes(buffer[0..8].try_into().unwrap()); + assert_eq!(null_bitset_0, 0, "Row 0 should have no nulls"); + let val0_0 = i64::from_le_bytes(buffer[8..16].try_into().unwrap()); + assert_eq!(val0_0, 1, "Row 0, col 0 should be 1"); + let val0_1 = i64::from_le_bytes(buffer[16..24].try_into().unwrap()); + assert_eq!(val0_1, 100, "Row 0, col 1 should be 100"); + let val0_2 = f64::from_bits(u64::from_le_bytes(buffer[24..32].try_into().unwrap())); + assert!((val0_2 - 1.5).abs() < 0.001, "Row 0, col 2 should be 1.5"); + + // Row 1: int32=null, int64=200 (not null), float64=2.5 (not null) + let null_bitset_1 = i64::from_le_bytes(buffer[32..40].try_into().unwrap()); + assert_eq!(null_bitset_1 & 1, 1, "Row 1, col 0 should be null"); + let val1_1 = i64::from_le_bytes(buffer[48..56].try_into().unwrap()); + assert_eq!(val1_1, 200, "Row 1, col 1 should be 200"); + + // Row 2: int32=3 (not null), int64=null, float64=3.5 (not null) + let null_bitset_2 = i64::from_le_bytes(buffer[64..72].try_into().unwrap()); + assert_eq!(null_bitset_2 & 2, 2, "Row 2, col 1 should be null"); + let val2_0 = i64::from_le_bytes(buffer[72..80].try_into().unwrap()); + assert_eq!(val2_0, 3, "Row 2, col 0 should be 3"); + } + + #[test] + fn test_mixed_schema_uses_general_path() { + // Test that schemas with variable-length types use the general path + let schema = vec![DataType::Int32, DataType::Utf8]; + let ctx = ColumnarToRowContext::new(schema, 100); + + // Should NOT be detected as all fixed-width + assert!( + !ctx.all_fixed_width, + "Schema with Utf8 should not be all fixed-width" + ); + } + + #[test] + fn test_convert_string_array() { + let schema = vec![DataType::Utf8]; + let mut ctx = ColumnarToRowContext::new(schema, 100); + + let array: ArrayRef = Arc::new(StringArray::from(vec!["hello", "world"])); + let arrays = vec![array]; + + let (ptr, offsets, lengths) = ctx.convert(&arrays, 2).unwrap(); + + assert!(!ptr.is_null()); + assert_eq!(offsets.len(), 2); + assert_eq!(lengths.len(), 2); + + // Row 0: 8 (bitset) + 8 (field slot) + 8 (aligned "hello") = 24 + // Row 1: 8 (bitset) + 8 (field slot) + 8 (aligned "world") = 24 + assert_eq!(lengths[0], 24); + assert_eq!(lengths[1], 24); + } + + #[test] + fn test_i128_to_spark_decimal_bytes() { + // Test positive number + let bytes = i128_to_spark_decimal_bytes(12345); + assert!(bytes.len() <= 16); + + // Test negative number + let bytes = i128_to_spark_decimal_bytes(-12345); + assert!(bytes.len() <= 16); + + // Test zero + let bytes = i128_to_spark_decimal_bytes(0); + assert!(!bytes.is_empty()); + } + + #[test] + fn test_list_data_conversion() { + use arrow::datatypes::Field; + + // Create a list with elements [0, 1, 2, 3, 4] + let values = Int32Array::from(vec![0, 1, 2, 3, 4]); + let offsets = arrow::buffer::OffsetBuffer::new(vec![0, 5].into()); + + let list_field = Arc::new(Field::new("item", DataType::Int32, true)); + let list_array = ListArray::new(Arc::clone(&list_field), offsets, Arc::new(values), None); + + // Convert the list for row 0 using the new direct-write function + let mut buffer = Vec::new(); + write_list_to_buffer(&mut buffer, &list_array, 0, &list_field).expect("conversion failed"); + let result = &buffer; + + // UnsafeArrayData format for Int32: + // [0..8]: numElements = 5 + // [8..16]: null bitset (8 bytes for up to 64 elements) + // [16..20]: element 0 (4 bytes for Int32) + // [20..24]: element 1 (4 bytes for Int32) + // ... (total 20 bytes for 5 elements, rounded up to 24 for 8-byte alignment) + + let num_elements = i64::from_le_bytes(result[0..8].try_into().unwrap()); + assert_eq!(num_elements, 5, "should have 5 elements"); + + let bitset_width = ColumnarToRowContext::calculate_bitset_width(5); + assert_eq!(bitset_width, 8); + + // Read each element value (Int32 uses 4 bytes per element) + let element_size = 4; // Int32 + for i in 0..5 { + let slot_offset = 8 + bitset_width + i * element_size; + let value = + i32::from_le_bytes(result[slot_offset..slot_offset + 4].try_into().unwrap()); + assert_eq!(value, i as i32, "element {} should be {}", i, i); + } + } + + #[test] + fn test_list_data_conversion_multiple_rows() { + use arrow::datatypes::Field; + + // Create multiple lists: + // Row 0: [0] + // Row 1: [0, 1] + // Row 2: [0, 1, 2] + let values = Int32Array::from(vec![ + 0, // row 0 + 0, 1, // row 1 + 0, 1, 2, // row 2 + ]); + let offsets = arrow::buffer::OffsetBuffer::new(vec![0, 1, 3, 6].into()); + + let list_field = Arc::new(Field::new("item", DataType::Int32, true)); + let list_array = ListArray::new(Arc::clone(&list_field), offsets, Arc::new(values), None); + + // Test row 1 which has elements [0, 1] + let mut buffer = Vec::new(); + write_list_to_buffer(&mut buffer, &list_array, 1, &list_field).expect("conversion failed"); + let result = &buffer; + + let num_elements = i64::from_le_bytes(result[0..8].try_into().unwrap()); + assert_eq!(num_elements, 2, "row 1 should have 2 elements"); + + // Int32 uses 4 bytes per element + let element_size = 4; + let bitset_width = ColumnarToRowContext::calculate_bitset_width(2); + let slot0_offset = 8 + bitset_width; + let slot1_offset = slot0_offset + element_size; + + let value0 = i32::from_le_bytes(result[slot0_offset..slot0_offset + 4].try_into().unwrap()); + let value1 = i32::from_le_bytes(result[slot1_offset..slot1_offset + 4].try_into().unwrap()); + + assert_eq!(value0, 0, "row 1, element 0 should be 0"); + assert_eq!(value1, 1, "row 1, element 1 should be 1"); + + // Also verify that list slicing is working correctly + let list_values = list_array.value(1); + assert_eq!(list_values.len(), 2, "row 1 should have 2 elements"); + let int_arr = list_values.as_any().downcast_ref::().unwrap(); + assert_eq!(int_arr.value(0), 0, "row 1 value[0] via Arrow should be 0"); + assert_eq!(int_arr.value(1), 1, "row 1 value[1] via Arrow should be 1"); + } + + #[test] + fn test_map_data_conversion() { + use arrow::datatypes::{Field, Fields}; + + // Create a map with 3 entries: {"key_0": 0, "key_1": 10, "key_2": 20} + let keys = StringArray::from(vec!["key_0", "key_1", "key_2"]); + let values = Int32Array::from(vec![0, 10, 20]); + + let entries_field = Field::new( + "entries", + DataType::Struct(Fields::from(vec![ + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Int32, true), + ])), + false, + ); + + let entries = StructArray::from(vec![ + ( + Arc::new(Field::new("key", DataType::Utf8, false)), + Arc::new(keys) as ArrayRef, + ), + ( + Arc::new(Field::new("value", DataType::Int32, true)), + Arc::new(values) as ArrayRef, + ), + ]); + + let map_array = MapArray::new( + Arc::new(entries_field.clone()), + arrow::buffer::OffsetBuffer::new(vec![0, 3].into()), + entries, + None, + false, + ); + + // Convert the map for row 0 + let mut buffer = Vec::new(); + write_map_to_buffer(&mut buffer, &map_array, 0, &Arc::new(entries_field)) + .expect("conversion failed"); + let result = &buffer; + + // Verify the structure: + // - [0..8]: key array size + // - [8..key_end]: key array data (UnsafeArrayData format) + // - [key_end..]: value array data (UnsafeArrayData format) + + let key_array_size = i64::from_le_bytes(result[0..8].try_into().unwrap()); + assert!(key_array_size > 0, "key array size should be positive"); + + let value_array_start = (8 + key_array_size) as usize; + assert!( + value_array_start < result.len(), + "value array should start within buffer" + ); + + // Read value array + let value_num_elements = i64::from_le_bytes( + result[value_array_start..value_array_start + 8] + .try_into() + .unwrap(), + ); + assert_eq!(value_num_elements, 3, "should have 3 values"); + + // Value array layout for Int32 (4 bytes per element): + // [0..8]: numElements = 3 + // [8..16]: null bitset (8 bytes for up to 64 elements) + // [16..20]: element 0 (4 bytes) + // [20..24]: element 1 (4 bytes) + // [24..28]: element 2 (4 bytes) + let value_bitset_width = ColumnarToRowContext::calculate_bitset_width(3); + assert_eq!(value_bitset_width, 8); + + let element_size = 4; // Int32 + let slot0_offset = value_array_start + 8 + value_bitset_width; + let slot1_offset = slot0_offset + element_size; + let slot2_offset = slot1_offset + element_size; + + let value0 = i32::from_le_bytes(result[slot0_offset..slot0_offset + 4].try_into().unwrap()); + let value1 = i32::from_le_bytes(result[slot1_offset..slot1_offset + 4].try_into().unwrap()); + let value2 = i32::from_le_bytes(result[slot2_offset..slot2_offset + 4].try_into().unwrap()); + + assert_eq!(value0, 0, "first value should be 0"); + assert_eq!(value1, 10, "second value should be 10"); + assert_eq!(value2, 20, "third value should be 20"); + } + + #[test] + fn test_map_data_conversion_multiple_rows() { + use arrow::datatypes::{Field, Fields}; + + // Create multiple maps: + // Row 0: {"key_0": 0} + // Row 1: {"key_0": 0, "key_1": 10} + // Row 2: {"key_0": 0, "key_1": 10, "key_2": 20} + // All entries are concatenated in the underlying arrays + let keys = StringArray::from(vec![ + "key_0", // row 0 + "key_0", "key_1", // row 1 + "key_0", "key_1", "key_2", // row 2 + ]); + let values = Int32Array::from(vec![ + 0, // row 0 + 0, 10, // row 1 + 0, 10, 20, // row 2 + ]); + + let entries_field = Field::new( + "entries", + DataType::Struct(Fields::from(vec![ + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Int32, true), + ])), + false, + ); + + let entries = StructArray::from(vec![ + ( + Arc::new(Field::new("key", DataType::Utf8, false)), + Arc::new(keys) as ArrayRef, + ), + ( + Arc::new(Field::new("value", DataType::Int32, true)), + Arc::new(values) as ArrayRef, + ), + ]); + + // Offsets: row 0 has 1 entry, row 1 has 2 entries, row 2 has 3 entries + let map_array = MapArray::new( + Arc::new(entries_field.clone()), + arrow::buffer::OffsetBuffer::new(vec![0, 1, 3, 6].into()), + entries, + None, + false, + ); + + // Test row 1 which has 2 entries + let mut buffer = Vec::new(); + write_map_to_buffer(&mut buffer, &map_array, 1, &Arc::new(entries_field.clone())) + .expect("conversion failed"); + let result = &buffer; + + let key_array_size = i64::from_le_bytes(result[0..8].try_into().unwrap()); + let value_array_start = (8 + key_array_size) as usize; + + let value_num_elements = i64::from_le_bytes( + result[value_array_start..value_array_start + 8] + .try_into() + .unwrap(), + ); + assert_eq!(value_num_elements, 2, "row 1 should have 2 values"); + + // Int32 uses 4 bytes per element + let element_size = 4; + let value_bitset_width = ColumnarToRowContext::calculate_bitset_width(2); + let slot0_offset = value_array_start + 8 + value_bitset_width; + let slot1_offset = slot0_offset + element_size; + + let value0 = i32::from_le_bytes(result[slot0_offset..slot0_offset + 4].try_into().unwrap()); + let value1 = i32::from_le_bytes(result[slot1_offset..slot1_offset + 4].try_into().unwrap()); + + assert_eq!(value0, 0, "row 1, first value should be 0"); + assert_eq!(value1, 10, "row 1, second value should be 10"); + + // Also verify that entries slicing is working correctly + let entries_row1 = map_array.value(1); + assert_eq!(entries_row1.len(), 2, "row 1 should have 2 entries"); + + let entries_values = entries_row1.column(1); + assert_eq!( + entries_values.len(), + 2, + "row 1 values should have 2 elements" + ); + + // Check the actual values from the sliced array + let values_arr = entries_values + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + values_arr.value(0), + 0, + "row 1 value[0] via Arrow should be 0" + ); + assert_eq!( + values_arr.value(1), + 10, + "row 1 value[1] via Arrow should be 10" + ); + } + + /// Test map conversion with a sliced MapArray to simulate FFI import behavior. + /// When data comes from FFI, the MapArray might be a slice of a larger array, + /// and the entries' child arrays might have offsets that don't start at 0. + #[test] + fn test_map_data_conversion_sliced_maparray() { + use arrow::datatypes::{Field, Fields}; + + // Create multiple maps (same as above) + let keys = StringArray::from(vec![ + "key_0", // row 0 + "key_0", "key_1", // row 1 + "key_0", "key_1", "key_2", // row 2 + ]); + let values = Int32Array::from(vec![ + 0, // row 0 + 0, 10, // row 1 + 0, 10, 20, // row 2 + ]); + + let entries_field = Field::new( + "entries", + DataType::Struct(Fields::from(vec![ + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Int32, true), + ])), + false, + ); + + let entries = StructArray::from(vec![ + ( + Arc::new(Field::new("key", DataType::Utf8, false)), + Arc::new(keys) as ArrayRef, + ), + ( + Arc::new(Field::new("value", DataType::Int32, true)), + Arc::new(values) as ArrayRef, + ), + ]); + + let map_array = MapArray::new( + Arc::new(entries_field.clone()), + arrow::buffer::OffsetBuffer::new(vec![0, 1, 3, 6].into()), + entries, + None, + false, + ); + + // Slice the MapArray to skip row 0 - this simulates what might happen with FFI + let sliced_map = map_array.slice(1, 2); + let sliced_map_array = sliced_map.as_any().downcast_ref::().unwrap(); + + // Now test row 0 of the sliced array (which is row 1 of the original) + let mut buffer = Vec::new(); + write_map_to_buffer( + &mut buffer, + sliced_map_array, + 0, + &Arc::new(entries_field.clone()), + ) + .expect("conversion failed"); + let result = &buffer; + + let key_array_size = i64::from_le_bytes(result[0..8].try_into().unwrap()); + let value_array_start = (8 + key_array_size) as usize; + + let value_num_elements = i64::from_le_bytes( + result[value_array_start..value_array_start + 8] + .try_into() + .unwrap(), + ); + assert_eq!(value_num_elements, 2, "sliced row 0 should have 2 values"); + + let value_bitset_width = ColumnarToRowContext::calculate_bitset_width(2); + let slot0_offset = value_array_start + 8 + value_bitset_width; + let slot1_offset = slot0_offset + 4; // Int32 uses 4 bytes + + let value0 = i32::from_le_bytes(result[slot0_offset..slot0_offset + 4].try_into().unwrap()); + let value1 = i32::from_le_bytes(result[slot1_offset..slot1_offset + 4].try_into().unwrap()); + + assert_eq!(value0, 0, "sliced row 0, first value should be 0"); + assert_eq!(value1, 10, "sliced row 0, second value should be 10"); + } + + #[test] + fn test_large_list_data_conversion() { + use arrow::datatypes::Field; + + // Create a large list with elements [0, 1, 2, 3, 4] + // LargeListArray uses i64 offsets instead of i32 + let values = Int32Array::from(vec![0, 1, 2, 3, 4]); + let offsets = arrow::buffer::OffsetBuffer::new(vec![0i64, 5].into()); + + let list_field = Arc::new(Field::new("item", DataType::Int32, true)); + let list_array = + LargeListArray::new(Arc::clone(&list_field), offsets, Arc::new(values), None); + + // Convert the list for row 0 + let mut buffer = Vec::new(); + write_large_list_to_buffer(&mut buffer, &list_array, 0, &list_field) + .expect("conversion failed"); + let result = &buffer; + + // UnsafeArrayData format for Int32: + // [0..8]: numElements = 5 + // [8..16]: null bitset (8 bytes for up to 64 elements) + // [16..20]: element 0 (4 bytes for Int32) + // ... (total 20 bytes for 5 elements, rounded up to 24 for 8-byte alignment) + + let num_elements = i64::from_le_bytes(result[0..8].try_into().unwrap()); + assert_eq!(num_elements, 5, "should have 5 elements"); + + let bitset_width = ColumnarToRowContext::calculate_bitset_width(5); + assert_eq!(bitset_width, 8); + + // Read each element value (Int32 uses 4 bytes per element) + let element_size = 4; // Int32 + for i in 0..5 { + let slot_offset = 8 + bitset_width + i * element_size; + let value = + i32::from_le_bytes(result[slot_offset..slot_offset + 4].try_into().unwrap()); + assert_eq!(value, i as i32, "element {} should be {}", i, i); + } + } + + #[test] + fn test_convert_fixed_size_binary_array() { + // FixedSizeBinary(3) - each value is exactly 3 bytes + let schema = vec![DataType::FixedSizeBinary(3)]; + let mut ctx = ColumnarToRowContext::new(schema, 100); + + let array: ArrayRef = Arc::new(FixedSizeBinaryArray::from(vec![ + Some(&[1u8, 2, 3][..]), + Some(&[4u8, 5, 6][..]), + None, // Test null handling + ])); + let arrays = vec![array]; + + let (ptr, offsets, lengths) = ctx.convert(&arrays, 3).unwrap(); + + assert!(!ptr.is_null()); + assert_eq!(offsets.len(), 3); + assert_eq!(lengths.len(), 3); + + // Row 0: 8 (bitset) + 8 (field slot) + 8 (aligned 3-byte data) = 24 + // Row 1: 8 (bitset) + 8 (field slot) + 8 (aligned 3-byte data) = 24 + // Row 2: 8 (bitset) + 8 (field slot) = 16 (null, no variable data) + assert_eq!(lengths[0], 24); + assert_eq!(lengths[1], 24); + assert_eq!(lengths[2], 16); + + // Verify the data is correct for non-null rows + unsafe { + let row0 = + std::slice::from_raw_parts(ptr.add(offsets[0] as usize), lengths[0] as usize); + // Variable data starts at offset 16 (8 bitset + 8 field slot) + assert_eq!(&row0[16..19], &[1u8, 2, 3]); + + let row1 = + std::slice::from_raw_parts(ptr.add(offsets[1] as usize), lengths[1] as usize); + assert_eq!(&row1[16..19], &[4u8, 5, 6]); + } + } + + #[test] + fn test_convert_dictionary_decimal_array() { + // Test that dictionary-encoded decimals are correctly unpacked and converted + // This tests the fix for casting to schema_type instead of value_type + use arrow::datatypes::Int8Type; + + // Create a dictionary array with Decimal128 values + // Values: [-0.01, -0.02, -0.03] represented as [-1, -2, -3] with scale 2 + let values = Decimal128Array::from(vec![-1i128, -2, -3]) + .with_precision_and_scale(5, 2) + .unwrap(); + + // Keys: [0, 1, 2, 0, 1, 2] - each value appears twice + let keys = Int8Array::from(vec![0i8, 1, 2, 0, 1, 2]); + + let dict_array: ArrayRef = + Arc::new(DictionaryArray::::try_new(keys, Arc::new(values)).unwrap()); + + // Schema expects Decimal128(5, 2) - not a dictionary type + let schema = vec![DataType::Decimal128(5, 2)]; + let mut ctx = ColumnarToRowContext::new(schema, 100); + + let arrays = vec![dict_array]; + let (ptr, offsets, lengths) = ctx.convert(&arrays, 6).unwrap(); + + assert!(!ptr.is_null()); + assert_eq!(offsets.len(), 6); + assert_eq!(lengths.len(), 6); + + // Verify the decimal values are correct (not doubled or otherwise corrupted) + // Fixed-width decimal is stored directly in the 8-byte field slot + unsafe { + for (i, expected) in [-1i64, -2, -3, -1, -2, -3].iter().enumerate() { + let row = + std::slice::from_raw_parts(ptr.add(offsets[i] as usize), lengths[i] as usize); + // Field value starts at offset 8 (after null bitset) + let value = i64::from_le_bytes(row[8..16].try_into().unwrap()); + assert_eq!( + value, *expected, + "Row {} should have value {}, got {}", + i, expected, value + ); + } + } + } + + #[test] + fn test_convert_int32_to_decimal128() { + // Test that Int32 arrays are correctly cast to Decimal128 when schema expects Decimal128. + // This can happen when COMET_USE_DECIMAL_128 is false and the parquet reader produces + // Int32 for small-precision decimals. + + // Create an Int32 array representing decimals: [-1, -2, -3] which at scale 2 means + // [-0.01, -0.02, -0.03] + let int_array: ArrayRef = Arc::new(Int32Array::from(vec![-1i32, -2, -3])); + + // Schema expects Decimal128(5, 2) + let schema = vec![DataType::Decimal128(5, 2)]; + let mut ctx = ColumnarToRowContext::new(schema, 100); + + let arrays = vec![int_array]; + let (ptr, offsets, lengths) = ctx.convert(&arrays, 3).unwrap(); + + assert!(!ptr.is_null()); + assert_eq!(offsets.len(), 3); + assert_eq!(lengths.len(), 3); + + // Verify the decimal values are correct after casting + // Fixed-width decimal is stored directly in the 8-byte field slot + unsafe { + for (i, expected) in [-1i64, -2, -3].iter().enumerate() { + let row = + std::slice::from_raw_parts(ptr.add(offsets[i] as usize), lengths[i] as usize); + // Field value starts at offset 8 (after null bitset) + let value = i64::from_le_bytes(row[8..16].try_into().unwrap()); + assert_eq!( + value, *expected, + "Row {} should have value {}, got {}", + i, expected, value + ); + } + } + } + + #[test] + fn test_convert_int64_to_decimal128() { + // Test that Int64 arrays are correctly cast to Decimal128 when schema expects Decimal128. + // This can happen when COMET_USE_DECIMAL_128 is false and the parquet reader produces + // Int64 for medium-precision decimals. + + // Create an Int64 array representing decimals + let int_array: ArrayRef = Arc::new(Int64Array::from(vec![-100i64, -200, -300])); + + // Schema expects Decimal128(10, 2) + let schema = vec![DataType::Decimal128(10, 2)]; + let mut ctx = ColumnarToRowContext::new(schema, 100); + + let arrays = vec![int_array]; + let (ptr, offsets, lengths) = ctx.convert(&arrays, 3).unwrap(); + + assert!(!ptr.is_null()); + assert_eq!(offsets.len(), 3); + assert_eq!(lengths.len(), 3); + + // Verify the decimal values are correct after casting + unsafe { + for (i, expected) in [-100i64, -200, -300].iter().enumerate() { + let row = + std::slice::from_raw_parts(ptr.add(offsets[i] as usize), lengths[i] as usize); + // Field value starts at offset 8 (after null bitset) + let value = i64::from_le_bytes(row[8..16].try_into().unwrap()); + assert_eq!( + value, *expected, + "Row {} should have value {}, got {}", + i, expected, value + ); + } + } + } +} diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 680cf80c75..e9f2d6523d 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -828,3 +828,121 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_logMemoryUsage( Ok(()) }) } + +// ============================================================================ +// Native Columnar to Row Conversion +// ============================================================================ + +use crate::execution::columnar_to_row::ColumnarToRowContext; +use arrow::ffi::{from_ffi, FFI_ArrowArray, FFI_ArrowSchema}; + +/// Initialize a native columnar to row converter. +/// +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers passed from JNI. +#[no_mangle] +pub unsafe extern "system" fn Java_org_apache_comet_Native_columnarToRowInit( + e: JNIEnv, + _class: JClass, + serialized_schema: JObjectArray, + batch_size: jint, +) -> jlong { + try_unwrap_or_throw(&e, |mut env| { + // Deserialize the schema + let schema = convert_datatype_arrays(&mut env, serialized_schema)?; + + // Create the context + let ctx = Box::new(ColumnarToRowContext::new(schema, batch_size as usize)); + + Ok(Box::into_raw(ctx) as jlong) + }) +} + +/// Convert Arrow columnar data to Spark UnsafeRow format. +/// +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers passed from JNI. +#[no_mangle] +pub unsafe extern "system" fn Java_org_apache_comet_Native_columnarToRowConvert( + e: JNIEnv, + _class: JClass, + c2r_handle: jlong, + array_addrs: JLongArray, + schema_addrs: JLongArray, + num_rows: jint, +) -> jni::sys::jobject { + try_unwrap_or_throw(&e, |mut env| { + // Get the context + let ctx = (c2r_handle as *mut ColumnarToRowContext) + .as_mut() + .ok_or_else(|| CometError::Internal("Null columnar to row context".to_string()))?; + + let num_cols = env.get_array_length(&array_addrs)? as usize; + + // Get array and schema addresses + let array_addrs_elements = env.get_array_elements(&array_addrs, ReleaseMode::NoCopyBack)?; + let schema_addrs_elements = + env.get_array_elements(&schema_addrs, ReleaseMode::NoCopyBack)?; + + // Import Arrow arrays from FFI + let mut arrays = Vec::with_capacity(num_cols); + for i in 0..num_cols { + let array_ptr = array_addrs_elements[i] as *mut FFI_ArrowArray; + let schema_ptr = schema_addrs_elements[i] as *mut FFI_ArrowSchema; + + // Take ownership of the FFI structures + let ffi_array = std::ptr::read(array_ptr); + let ffi_schema = std::ptr::read(schema_ptr); + + // Convert to Arrow ArrayData + let array_data = from_ffi(ffi_array, &ffi_schema) + .map_err(|e| CometError::Internal(format!("Failed to import array: {}", e)))?; + + arrays.push(arrow::array::make_array(array_data)); + } + + // Convert columnar to row + let (buffer_ptr, offsets, lengths) = ctx.convert(&arrays, num_rows as usize)?; + + // Create Java int arrays for offsets and lengths + let offsets_array = env.new_int_array(offsets.len() as i32)?; + env.set_int_array_region(&offsets_array, 0, offsets)?; + + let lengths_array = env.new_int_array(lengths.len() as i32)?; + env.set_int_array_region(&lengths_array, 0, lengths)?; + + // Create the NativeColumnarToRowInfo object + let info_class = env.find_class("org/apache/comet/NativeColumnarToRowInfo")?; + let info_obj = env.new_object( + info_class, + "(J[I[I)V", + &[ + jni::objects::JValue::Long(buffer_ptr as jlong), + jni::objects::JValue::Object(&offsets_array), + jni::objects::JValue::Object(&lengths_array), + ], + )?; + + Ok(info_obj.into_raw()) + }) +} + +/// Close and release the native columnar to row converter. +/// +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers passed from JNI. +#[no_mangle] +pub unsafe extern "system" fn Java_org_apache_comet_Native_columnarToRowClose( + e: JNIEnv, + _class: JClass, + c2r_handle: jlong, +) { + try_unwrap_or_throw(&e, |_env| { + if c2r_handle != 0 { + let _ctx: Box = + Box::from_raw(c2r_handle as *mut ColumnarToRowContext); + // ctx is dropped here, freeing the buffer + } + Ok(()) + }) +} diff --git a/native/core/src/execution/mod.rs b/native/core/src/execution/mod.rs index 06b5585582..85fc672461 100644 --- a/native/core/src/execution/mod.rs +++ b/native/core/src/execution/mod.rs @@ -16,6 +16,7 @@ // under the License. //! PoC of vectorization execution through JNI to Rust. +pub mod columnar_to_row; pub mod expressions; pub mod jni_api; pub(crate) mod metrics; diff --git a/spark/src/main/java/org/apache/comet/NativeColumnarToRowInfo.java b/spark/src/main/java/org/apache/comet/NativeColumnarToRowInfo.java new file mode 100644 index 0000000000..319390255b --- /dev/null +++ b/spark/src/main/java/org/apache/comet/NativeColumnarToRowInfo.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet; + +/** + * Container for the result of native columnar to row conversion. + * + *

This class holds the memory address of the converted row data buffer and metadata about each + * row (offsets and lengths). The native side allocates and owns the memory buffer, and this class + * provides the JVM with the information needed to read the UnsafeRow data. + * + *

Memory Layout of the buffer: + * + *

+ * ┌─────────────────────────────────────────────────────────────┐
+ * │ Row 0: [null bitset][fixed-width values][variable-length]  │
+ * ├─────────────────────────────────────────────────────────────┤
+ * │ Row 1: [null bitset][fixed-width values][variable-length]  │
+ * ├─────────────────────────────────────────────────────────────┤
+ * │ ...                                                         │
+ * └─────────────────────────────────────────────────────────────┘
+ * 
+ * + *

The offsets array provides the byte offset from memoryAddress where each row starts. The + * lengths array provides the total byte length of each row. + */ +public class NativeColumnarToRowInfo { + /** The memory address of the buffer containing all converted rows. */ + public final long memoryAddress; + + /** The byte offset from memoryAddress where each row starts. */ + public final int[] offsets; + + /** The total byte length of each row. */ + public final int[] lengths; + + /** + * Constructs a NativeColumnarToRowInfo with the given memory address and row metadata. + * + * @param memoryAddress The memory address of the buffer containing converted row data. + * @param offsets The byte offset for each row from the base memory address. + * @param lengths The byte length of each row. + */ + public NativeColumnarToRowInfo(long memoryAddress, int[] offsets, int[] lengths) { + this.memoryAddress = memoryAddress; + this.offsets = offsets; + this.lengths = lengths; + } + + /** + * Returns the number of rows in this result. + * + * @return The number of rows. + */ + public int numRows() { + return offsets != null ? offsets.length : 0; + } +} diff --git a/spark/src/main/scala/org/apache/comet/ExtendedExplainInfo.scala b/spark/src/main/scala/org/apache/comet/ExtendedExplainInfo.scala index 56ae64ed68..f47428e801 100644 --- a/spark/src/main/scala/org/apache/comet/ExtendedExplainInfo.scala +++ b/spark/src/main/scala/org/apache/comet/ExtendedExplainInfo.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import org.apache.spark.sql.ExtendedExplainGenerator import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} -import org.apache.spark.sql.comet.{CometColumnarToRowExec, CometPlan, CometSparkToColumnarExec} +import org.apache.spark.sql.comet.{CometColumnarToRowExec, CometNativeColumnarToRowExec, CometPlan, CometSparkToColumnarExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, InputAdapter, RowToColumnarExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, QueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec @@ -104,7 +104,7 @@ class ExtendedExplainInfo extends ExtendedExplainGenerator { _: WholeStageCodegenExec | _: ReusedExchangeExec | _: AQEShuffleReadExec => // ignore case _: RowToColumnarExec | _: ColumnarToRowExec | _: CometColumnarToRowExec | - _: CometSparkToColumnarExec => + _: CometNativeColumnarToRowExec | _: CometSparkToColumnarExec => planStats.transitions += 1 case _: CometPlan => planStats.cometOperators += 1 diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index 7f8131ff7c..55e0c70e72 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -203,4 +203,49 @@ class Native extends NativeBase { */ @native def logMemoryUsage(name: String, memoryUsageBytes: Long): Unit + // Native Columnar to Row conversion methods + + /** + * Initialize a native columnar to row converter. + * + * @param schema + * Array of serialized data types (as byte arrays) for each column in the schema. + * @param batchSize + * The maximum number of rows that will be converted in a single batch. Used to pre-allocate + * the output buffer. + * @return + * A handle to the native converter context. This handle must be passed to subsequent convert + * and close calls. + */ + @native def columnarToRowInit(schema: Array[Array[Byte]], batchSize: Int): Long + + /** + * Convert Arrow columnar data to Spark UnsafeRow format. + * + * @param c2rHandle + * The handle returned by columnarToRowInit. + * @param arrayAddrs + * The addresses of Arrow Array structures for each column. + * @param schemaAddrs + * The addresses of Arrow Schema structures for each column. + * @param numRows + * The number of rows to convert. + * @return + * A NativeColumnarToRowInfo containing the memory address of the row buffer and metadata + * (offsets and lengths) for each row. + */ + @native def columnarToRowConvert( + c2rHandle: Long, + arrayAddrs: Array[Long], + schemaAddrs: Array[Long], + numRows: Int): NativeColumnarToRowInfo + + /** + * Close and release the native columnar to row converter. + * + * @param c2rHandle + * The handle returned by columnarToRowInit. + */ + @native def columnarToRowClose(c2rHandle: Long): Unit + } diff --git a/spark/src/main/scala/org/apache/comet/NativeColumnarToRowConverter.scala b/spark/src/main/scala/org/apache/comet/NativeColumnarToRowConverter.scala new file mode 100644 index 0000000000..364df5f4c9 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/NativeColumnarToRowConverter.scala @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +import org.apache.comet.serde.QueryPlanSerde +import org.apache.comet.vector.NativeUtil + +/** + * Native converter that converts Arrow columnar data to Spark UnsafeRow format. + * + * This converter maintains a native handle that holds the conversion context and output buffer. + * The buffer is reused across conversions to minimize allocations. + * + * Memory Management: + * - The native side owns the output buffer + * - UnsafeRow objects returned by convert() point directly to native memory (zero-copy) + * - The buffer is valid until the next convert() call or close() + * - Always call close() when done to release native resources + * + * @param schema + * The schema of the data to convert + * @param batchSize + * Maximum number of rows per batch (used for buffer pre-allocation) + */ +class NativeColumnarToRowConverter(schema: StructType, batchSize: Int) extends AutoCloseable { + + private val nativeLib = new Native() + private val nativeUtil = new NativeUtil() + + // Serialize the schema for native initialization + private val serializedSchema: Array[Array[Byte]] = schema.fields.map { field => + QueryPlanSerde.serializeDataType(field.dataType) match { + case Some(dataType) => dataType.toByteArray + case None => + throw new UnsupportedOperationException( + s"Data type ${field.dataType} is not supported for native columnar to row conversion") + } + } + + // Initialize native context - handle is 0 if initialization failed + private var c2rHandle: Long = nativeLib.columnarToRowInit(serializedSchema, batchSize) + + // Reusable UnsafeRow for iteration + private val unsafeRow = new UnsafeRow(schema.fields.length) + + /** + * Converts a ColumnarBatch to an iterator of InternalRows. + * + * The returned iterator yields UnsafeRow objects that point directly to native memory. These + * rows are valid only until the next call to convert() or close(). + * + * @param batch + * The columnar batch to convert + * @return + * An iterator of InternalRows + */ + def convert(batch: ColumnarBatch): Iterator[InternalRow] = { + if (c2rHandle == 0) { + throw new IllegalStateException("NativeColumnarToRowConverter has been closed") + } + + val numRows = batch.numRows() + if (numRows == 0) { + return Iterator.empty + } + + // Export the batch to Arrow FFI and get memory addresses + val (arrayAddrs, schemaAddrs, exportedNumRows) = nativeUtil.exportBatchToAddresses(batch) + + // Call native conversion + val info = nativeLib.columnarToRowConvert(c2rHandle, arrayAddrs, schemaAddrs, exportedNumRows) + + // Return an iterator that yields UnsafeRows pointing to native memory + new NativeRowIterator(info, unsafeRow) + } + + /** + * Checks if this converter is still open and usable. + */ + def isOpen: Boolean = c2rHandle != 0 + + /** + * Closes the converter and releases native resources. + */ + override def close(): Unit = { + if (c2rHandle != 0) { + nativeLib.columnarToRowClose(c2rHandle) + c2rHandle = 0 + } + nativeUtil.close() + } +} + +/** + * Iterator that yields UnsafeRows backed by native memory. + * + * The UnsafeRow is reused across iterations - callers must copy the row if they need to retain it + * beyond the current iteration. + */ +private class NativeRowIterator(info: NativeColumnarToRowInfo, unsafeRow: UnsafeRow) + extends Iterator[InternalRow] { + + private var currentIdx = 0 + private val numRows = info.numRows() + + override def hasNext: Boolean = currentIdx < numRows + + override def next(): InternalRow = { + if (!hasNext) { + throw new NoSuchElementException("No more rows") + } + + // Point the UnsafeRow to the native memory + val rowAddress = info.memoryAddress + info.offsets(currentIdx) + val rowSize = info.lengths(currentIdx) + + unsafeRow.pointTo(null, rowAddress, rowSize) + currentIdx += 1 + + unsafeRow + } +} diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala index bf0ac324cf..d1c3b07677 100644 --- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala +++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala @@ -22,13 +22,14 @@ package org.apache.comet.rules import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.sideBySide -import org.apache.spark.sql.comet.{CometCollectLimitExec, CometColumnarToRowExec, CometNativeWriteExec, CometPlan, CometSparkToColumnarExec} +import org.apache.spark.sql.comet.{CometBatchScanExec, CometCollectLimitExec, CometColumnarToRowExec, CometNativeColumnarToRowExec, CometNativeWriteExec, CometPlan, CometScanExec, CometSparkToColumnarExec} import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.comet.CometConf +import org.apache.comet.parquet.CometParquetScan // This rule is responsible for eliminating redundant transitions between row-based and // columnar-based operators for Comet. Currently, three potential redundant transitions are: @@ -85,7 +86,7 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa case ColumnarToRowExec(nativeWrite: CometNativeWriteExec) => nativeWrite case c @ ColumnarToRowExec(child) if hasCometNativeChild(child) => - val op = CometColumnarToRowExec(child) + val op = createColumnarToRowExec(child) if (c.logicalLink.isEmpty) { op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) @@ -95,6 +96,8 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa op case CometColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => sparkToColumnar.child + case CometNativeColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => + sparkToColumnar.child case CometSparkToColumnarExec(child: CometSparkToColumnarExec) => child // Spark adds `RowToColumnar` under Comet columnar shuffle. But it's redundant as the // shuffle takes row-based input. @@ -113,6 +116,8 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa child case CometColumnarToRowExec(child: CometCollectLimitExec) => child + case CometNativeColumnarToRowExec(child: CometCollectLimitExec) => + child case other => other } @@ -125,4 +130,49 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa case _ => op.exists(_.isInstanceOf[CometPlan]) } } + + /** + * Creates an appropriate columnar to row transition operator. + * + * If native columnar to row conversion is enabled and the schema is supported, uses + * CometNativeColumnarToRowExec. Otherwise falls back to CometColumnarToRowExec. + */ + private def createColumnarToRowExec(child: SparkPlan): SparkPlan = { + val schema = child.schema + val useNative = CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.get() && + CometNativeColumnarToRowExec.supportsSchema(schema) && + !hasScanUsingMutableBuffers(child) + + if (useNative) { + CometNativeColumnarToRowExec(child) + } else { + CometColumnarToRowExec(child) + } + } + + /** + * Checks if the plan contains a scan that uses mutable buffers. Native C2R is not compatible + * with such scans because the buffers may be modified after C2R reads them. + * + * This includes: + * - CometScanExec with native_comet scan implementation (V1 path) - uses BatchReader + * - CometScanExec with native_iceberg_compat and partition columns - uses + * ConstantColumnReader + * - CometBatchScanExec with CometParquetScan (V2 Parquet path) - uses BatchReader + */ + private def hasScanUsingMutableBuffers(op: SparkPlan): Boolean = { + op match { + case c: QueryStageExec => hasScanUsingMutableBuffers(c.plan) + case c: ReusedExchangeExec => hasScanUsingMutableBuffers(c.child) + case _ => + op.exists { + case scan: CometScanExec => + scan.scanImpl == CometConf.SCAN_NATIVE_COMET || + (scan.scanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT && + scan.relation.partitionSchema.nonEmpty) + case scan: CometBatchScanExec => scan.scan.isInstanceOf[CometParquetScan] + case _ => false + } + } + } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeColumnarToRowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeColumnarToRowExec.scala new file mode 100644 index 0000000000..dabc548384 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeColumnarToRowExec.scala @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.comet + +import java.util.UUID +import java.util.concurrent.{Future, TimeoutException, TimeUnit} + +import scala.concurrent.Promise +import scala.util.control.NonFatal + +import org.apache.spark.{broadcast, SparkException, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.comet.util.{Utils => CometUtils} +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.{ColumnarToRowTransition, SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.{SparkFatalException, Utils} + +import org.apache.comet.{CometConf, NativeColumnarToRowConverter} + +/** + * Native implementation of ColumnarToRowExec that converts Arrow columnar data to Spark UnsafeRow + * format using Rust. + * + * This is an experimental feature that can be enabled by setting + * `spark.comet.columnarToRow.native.enabled=true`. + * + * Benefits over the JVM implementation: + * - Zero-copy for variable-length types (strings, binary) + * - Better CPU cache utilization through vectorized processing + * - Reduced GC pressure + * + * @param child + * The child plan that produces columnar batches + */ +case class CometNativeColumnarToRowExec(child: SparkPlan) + extends ColumnarToRowTransition + with CometPlan { + + // supportsColumnar requires to be only called on driver side, see also SPARK-37779. + assert(Utils.isInRunningSparkTask || child.supportsColumnar) + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override lazy val metrics: Map[String, SQLMetric] = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numInputBatches" -> SQLMetrics.createMetric(sparkContext, "number of input batches"), + "convertTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time in conversion")) + + @transient + private lazy val promise = Promise[broadcast.Broadcast[Any]]() + + @transient + private val timeout: Long = conf.broadcastTimeout + + private val runId: UUID = UUID.randomUUID + + private lazy val cometBroadcastExchange = findCometBroadcastExchange(child) + + @transient + lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { + SQLExecution.withThreadLocalCaptured[broadcast.Broadcast[Any]]( + session, + CometBroadcastExchangeExec.executionContext) { + try { + // Setup a job group here so later it may get cancelled by groupId if necessary. + sparkContext.setJobGroup( + runId.toString, + s"CometNativeColumnarToRow broadcast exchange (runId $runId)", + interruptOnCancel = true) + + val numOutputRows = longMetric("numOutputRows") + val numInputBatches = longMetric("numInputBatches") + val localSchema = this.schema + val batchSize = CometConf.COMET_BATCH_SIZE.get() + val broadcastColumnar = child.executeBroadcast() + val serializedBatches = + broadcastColumnar.value.asInstanceOf[Array[org.apache.spark.util.io.ChunkedByteBuffer]] + + // Use native converter to convert columnar data to rows + val converter = new NativeColumnarToRowConverter(localSchema, batchSize) + try { + val rows = serializedBatches.iterator + .flatMap(CometUtils.decodeBatches(_, this.getClass.getSimpleName)) + .flatMap { batch => + numInputBatches += 1 + numOutputRows += batch.numRows() + converter.convert(batch) + } + + val mode = cometBroadcastExchange.get.mode + val relation = mode.transform(rows, Some(numOutputRows.value)) + val broadcasted = sparkContext.broadcastInternal(relation, serializedOnly = true) + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) + promise.trySuccess(broadcasted) + broadcasted + } finally { + converter.close() + } + } catch { + // SPARK-24294: To bypass scala bug: https://github.com/scala/bug/issues/9554, we throw + // SparkFatalException, which is a subclass of Exception. ThreadUtils.awaitResult + // will catch this exception and re-throw the wrapped fatal throwable. + case oe: OutOfMemoryError => + val ex = new SparkFatalException(oe) + promise.tryFailure(ex) + throw ex + case e if !NonFatal(e) => + val ex = new SparkFatalException(e) + promise.tryFailure(ex) + throw ex + case e: Throwable => + promise.tryFailure(e) + throw e + } + } + } + + override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + if (cometBroadcastExchange.isEmpty) { + throw new SparkException( + "CometNativeColumnarToRowExec only supports doExecuteBroadcast when child contains a " + + "CometBroadcastExchange, but got " + child) + } + + try { + relationFuture.get(timeout, TimeUnit.SECONDS).asInstanceOf[broadcast.Broadcast[T]] + } catch { + case ex: TimeoutException => + logError(s"Could not execute broadcast in $timeout secs.", ex) + if (!relationFuture.isDone) { + sparkContext.cancelJobGroup(runId.toString) + relationFuture.cancel(true) + } + throw QueryExecutionErrors.executeBroadcastTimeoutError(timeout, Some(ex)) + } + } + + private def findCometBroadcastExchange(op: SparkPlan): Option[CometBroadcastExchangeExec] = { + op match { + case b: CometBroadcastExchangeExec => Some(b) + case b: BroadcastQueryStageExec => findCometBroadcastExchange(b.plan) + case b: ReusedExchangeExec => findCometBroadcastExchange(b.child) + case _ => op.children.collectFirst(Function.unlift(findCometBroadcastExchange)) + } + } + + override def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + val numInputBatches = longMetric("numInputBatches") + val convertTime = longMetric("convertTime") + + // Get the schema and batch size for native conversion + val localSchema = child.schema + val batchSize = CometConf.COMET_BATCH_SIZE.get() + + child.executeColumnar().mapPartitionsInternal { batches => + // Create native converter for this partition + val converter = new NativeColumnarToRowConverter(localSchema, batchSize) + + // Register cleanup on task completion + TaskContext.get().addTaskCompletionListener[Unit] { _ => + converter.close() + } + + batches.flatMap { batch => + numInputBatches += 1 + val numRows = batch.numRows() + numOutputRows += numRows + + val startTime = System.nanoTime() + val result = converter.convert(batch) + convertTime += System.nanoTime() - startTime + + result + } + } + } + + override protected def withNewChildInternal(newChild: SparkPlan): CometNativeColumnarToRowExec = + copy(child = newChild) +} + +object CometNativeColumnarToRowExec { + + /** + * Checks if native columnar to row conversion is enabled. + */ + def isEnabled: Boolean = CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.get() + + /** + * Checks if the given schema is supported by native columnar to row conversion. + * + * Currently supported types: + * - Primitive types: Boolean, Byte, Short, Int, Long, Float, Double + * - Date and Timestamp (microseconds) + * - Decimal (both inline and variable-length) + * - String and Binary + * - Struct, Array, Map (nested types) + */ + def supportsSchema(schema: StructType): Boolean = { + import org.apache.spark.sql.types._ + + def isSupported(dataType: DataType): Boolean = dataType match { + case BooleanType | ByteType | ShortType | IntegerType | LongType => true + case FloatType | DoubleType => true + case DateType => true + case TimestampType => true + case _: DecimalType => true + case StringType | BinaryType => true + case StructType(fields) => fields.forall(f => isSupported(f.dataType)) + case ArrayType(elementType, _) => isSupported(elementType) + case MapType(keyType, valueType, _) => isSupported(keyType) && isSupported(valueType) + case _ => false + } + + schema.fields.forall(f => isSupported(f.dataType)) + } +} diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt index ba9b1ca1f6..5e3f8e7081 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (44) : : +- * BroadcastHashJoin Inner BuildRight (28) : : :- * Filter (11) : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometColumnarExchange (8) : : : +- * HashAggregate (7) : : : +- * Project (6) @@ -20,11 +20,11 @@ TakeOrderedAndProject (44) : : +- BroadcastExchange (27) : : +- * Filter (26) : : +- * HashAggregate (25) - : : +- * CometColumnarToRow (24) + : : +- CometNativeColumnarToRow (24) : : +- CometColumnarExchange (23) : : +- * HashAggregate (22) : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (20) + : : +- CometNativeColumnarToRow (20) : : +- CometColumnarExchange (19) : : +- * HashAggregate (18) : : +- * Project (17) @@ -34,12 +34,12 @@ TakeOrderedAndProject (44) : : : +- Scan parquet spark_catalog.default.store_returns (12) : : +- ReusedExchange (15) : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- CometProject (32) : +- CometFilter (31) : +- CometNativeScan parquet spark_catalog.default.store (30) +- BroadcastExchange (41) - +- * CometColumnarToRow (40) + +- CometNativeColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometNativeScan parquet spark_catalog.default.customer (37) @@ -53,27 +53,27 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 2] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [sr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 2] +(6) Project [codegen id : 1] Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -(7) HashAggregate [codegen id : 2] +(7) HashAggregate [codegen id : 1] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] @@ -84,17 +84,17 @@ Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 9] +(9) CometNativeColumnarToRow Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -(10) HashAggregate [codegen id : 9] +(10) HashAggregate [codegen id : 5] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(11) Filter [codegen id : 9] +(11) Filter [codegen id : 5] Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] Condition : isnotnull(ctr_total_return#12) @@ -106,27 +106,27 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 2] Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -(14) Filter [codegen id : 4] +(14) Filter [codegen id : 2] Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] Condition : isnotnull(sr_store_sk#14) (15) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#17] -(16) BroadcastHashJoin [codegen id : 4] +(16) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#16] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(17) Project [codegen id : 4] +(17) Project [codegen id : 2] Output [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#17] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] Keys [2]: [sr_customer_sk#13, sr_store_sk#14] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))] @@ -137,17 +137,17 @@ Results [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -(21) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 3] Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] Keys [2]: [sr_customer_sk#13, sr_store_sk#14] Functions [1]: [sum(UnscaledValue(sr_return_amt#15))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9] Results [2]: [sr_store_sk#14 AS ctr_store_sk#20, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#21] -(22) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 3] Input [2]: [ctr_store_sk#20, ctr_total_return#21] Keys [1]: [ctr_store_sk#20] Functions [1]: [partial_avg(ctr_total_return#21)] @@ -158,17 +158,17 @@ Results [3]: [ctr_store_sk#20, sum#24, count#25] Input [3]: [ctr_store_sk#20, sum#24, count#25] Arguments: hashpartitioning(ctr_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 6] +(24) CometNativeColumnarToRow Input [3]: [ctr_store_sk#20, sum#24, count#25] -(25) HashAggregate [codegen id : 6] +(25) HashAggregate [codegen id : 4] Input [3]: [ctr_store_sk#20, sum#24, count#25] Keys [1]: [ctr_store_sk#20] Functions [1]: [avg(ctr_total_return#21)] Aggregate Attributes [1]: [avg(ctr_total_return#21)#26] Results [2]: [(avg(ctr_total_return#21)#26 * 1.2) AS (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -(26) Filter [codegen id : 6] +(26) Filter [codegen id : 4] Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) @@ -176,13 +176,13 @@ Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] -(28) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_store_sk#11] Right keys [1]: [ctr_store_sk#20] Join type: Inner Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27) -(29) Project [codegen id : 9] +(29) Project [codegen id : 5] Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] @@ -201,20 +201,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#28, s_state#29] Arguments: [s_store_sk#28], [s_store_sk#28] -(33) CometColumnarToRow [codegen id : 7] +(33) CometNativeColumnarToRow Input [1]: [s_store_sk#28] (34) BroadcastExchange Input [1]: [s_store_sk#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(35) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_store_sk#11] Right keys [1]: [s_store_sk#28] Join type: Inner Join condition: None -(36) Project [codegen id : 9] +(36) Project [codegen id : 5] Output [1]: [ctr_customer_sk#10] Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#28] @@ -233,20 +233,20 @@ Condition : isnotnull(c_customer_sk#30) Input [2]: [c_customer_sk#30, c_customer_id#31] Arguments: [c_customer_sk#30, c_customer_id#32], [c_customer_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#31, 16, true, false, true) AS c_customer_id#32] -(40) CometColumnarToRow [codegen id : 8] +(40) CometNativeColumnarToRow Input [2]: [c_customer_sk#30, c_customer_id#32] (41) BroadcastExchange Input [2]: [c_customer_sk#30, c_customer_id#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(42) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_customer_sk#10] Right keys [1]: [c_customer_sk#30] Join type: Inner Join condition: None -(43) Project [codegen id : 9] +(43) Project [codegen id : 5] Output [1]: [c_customer_id#32] Input [3]: [ctr_customer_sk#10, c_customer_sk#30, c_customer_id#32] @@ -258,7 +258,7 @@ Arguments: 100, [c_customer_id#32 ASC NULLS FIRST], [c_customer_id#32] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometNativeScan parquet spark_catalog.default.date_dim (45) @@ -279,7 +279,7 @@ Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#6, d_year#33] Arguments: [d_date_sk#6], [d_date_sk#6] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt index 0622aad9d5..91b07a7538 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -17,23 +17,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -43,17 +43,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt index ed85c142aa..1351d8c439 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (9) + WholeStageCodegen (5) Project [c_customer_id] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk] @@ -8,10 +8,10 @@ TakeOrderedAndProject [c_customer_id] BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] Project [sr_customer_sk,sr_store_sk,sr_return_amt] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -21,29 +21,27 @@ TakeOrderedAndProject [c_customer_id] Scan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (6) + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ctr_store_sk] #4 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] Project [sr_customer_sk,sr_store_sk,sr_return_amt] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -56,17 +54,13 @@ TakeOrderedAndProject [c_customer_id] ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] + CometNativeColumnarToRow + CometProject [c_customer_id] [c_customer_sk,c_customer_id] + CometFilter [c_customer_sk,c_customer_id] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/explain.txt index e66401996f..5cb423b2f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/explain.txt @@ -236,7 +236,7 @@ Input [1]: [c_customer_id#28] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt index 8aa14c43dd..0d0d80c7f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt index 3e1d6243f8..d69c52dda0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt index e66401996f..5cb423b2f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt @@ -236,7 +236,7 @@ Input [1]: [c_customer_id#28] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/extended.txt index 8aa14c43dd..0d0d80c7f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 3e1d6243f8..d69c52dda0 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/explain.txt index 29034bd457..ae8af59601 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -13,7 +13,7 @@ TakeOrderedAndProject (45) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -35,12 +35,12 @@ TakeOrderedAndProject (45) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_address (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) @@ -57,7 +57,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] (6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] @@ -87,7 +87,7 @@ Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi @@ -100,19 +100,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] (13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] @@ -120,7 +120,7 @@ Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) @@ -133,19 +133,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] @@ -153,17 +153,17 @@ Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(25) Filter [codegen id : 4] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -182,20 +182,20 @@ Condition : (ca_county#17 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#16, ca_county#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16] @@ -214,24 +214,24 @@ Condition : isnotnull(cd_demo_sk#18) Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#29, cd_purchase_estimate#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#23, 10, true, false, true) AS cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] (38) BroadcastExchange Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 4] Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 4] Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [partial_count(1)] @@ -242,10 +242,10 @@ Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_pur Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 5] Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [count(1)] @@ -260,7 +260,7 @@ Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FI Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * CometColumnarToRow (49) ++- CometNativeColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometNativeScan parquet spark_catalog.default.date_dim (46) @@ -281,7 +281,7 @@ Condition : (((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = Input [3]: [d_date_sk#9, d_year#40, d_moy#41] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt index 8935b6cc13..40a8ef4a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -22,12 +22,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,17 +49,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt index 0cc108b0b0..ff26eb68a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -29,17 +29,15 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,7 +48,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -61,17 +59,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/explain.txt index 68bd670d8e..c11b7ccdb3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [1]: [ca_address_sk#20] (33) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#22) Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (40) BroadcastExchange Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] @@ -253,10 +253,10 @@ Results [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_pur Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] @@ -271,7 +271,7 @@ Arguments: 100, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt index e7193f87e1..fd519f33e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt index 68bd670d8e..c11b7ccdb3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [1]: [ca_address_sk#20] (33) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#22) Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (40) BroadcastExchange Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] @@ -253,10 +253,10 @@ Results [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_pur Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] @@ -271,7 +271,7 @@ Arguments: 100, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt index e7193f87e1..fd519f33e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/explain.txt index d697d27f18..300b750cd1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/explain.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject (80) : : +- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -26,14 +26,14 @@ TakeOrderedAndProject (80) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -45,14 +45,14 @@ TakeOrderedAndProject (80) : +- BroadcastExchange (57) : +- * Filter (56) : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) + : +- CometNativeColumnarToRow (54) : +- CometColumnarExchange (53) : +- * HashAggregate (52) : +- * Project (51) : +- * BroadcastHashJoin Inner BuildRight (50) : :- * Project (48) : : +- * BroadcastHashJoin Inner BuildRight (47) - : : :- * CometColumnarToRow (42) + : : :- CometNativeColumnarToRow (42) : : : +- CometProject (41) : : : +- CometFilter (40) : : : +- CometNativeScan parquet spark_catalog.default.customer (39) @@ -63,14 +63,14 @@ TakeOrderedAndProject (80) : +- ReusedExchange (49) +- BroadcastExchange (77) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- CometColumnarExchange (74) +- * HashAggregate (73) +- * Project (72) +- * BroadcastHashJoin Inner BuildRight (71) :- * Project (69) : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * CometColumnarToRow (63) + : :- CometNativeColumnarToRow (63) : : +- CometProject (62) : : +- CometFilter (61) : : +- CometNativeScan parquet spark_catalog.default.customer (60) @@ -96,7 +96,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] (5) Scan parquet spark_catalog.default.store_sales @@ -118,30 +118,30 @@ Condition : isnotnull(ss_customer_sk#15) Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#15] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] (11) ReusedExchange [Reuses operator id: 84] Output [2]: [d_date_sk#20, d_year#21] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] @@ -152,17 +152,17 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#25, year_total#26] Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) @@ -181,7 +181,7 @@ Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] (23) Scan parquet spark_catalog.default.store_sales @@ -192,10 +192,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Condition : isnotnull(ss_customer_sk#35) @@ -203,30 +203,30 @@ Condition : isnotnull(ss_customer_sk#35) Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#27] Right keys [1]: [ss_customer_sk#35] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] (29) ReusedExchange [Reuses operator id: 88] Output [2]: [d_date_sk#40, d_year#41] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#38] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] @@ -237,10 +237,10 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] @@ -251,13 +251,13 @@ Results [3]: [c_customer_id#9 AS customer_id#44, c_preferred_cust_flag#12 AS cus Input [3]: [customer_id#44, customer_preferred_cust_flag#45, year_total#46] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#44] Join type: Inner Join condition: None -(38) Project [codegen id : 16] +(38) Project [codegen id : 12] Output [4]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46] Input [5]: [customer_id#25, year_total#26, customer_id#44, customer_preferred_cust_flag#45, year_total#46] @@ -276,7 +276,7 @@ Condition : (isnotnull(c_customer_sk#47) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] Arguments: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60], [c_customer_sk#47, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#48, 16, true, false, true) AS c_customer_id#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#49, 20, true, false, true) AS c_first_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#50, 30, true, false, true) AS c_last_name#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#51, 1, true, false, true) AS c_preferred_cust_flag#58, c_birth_country#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#53, 13, true, false, true) AS c_login#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#54, 50, true, false, true) AS c_email_address#60] -(42) CometColumnarToRow [codegen id : 10] +(42) CometNativeColumnarToRow Input [8]: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60] (43) Scan parquet spark_catalog.default.web_sales @@ -287,10 +287,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 8] +(44) ColumnarToRow [codegen id : 6] Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -(45) Filter [codegen id : 8] +(45) Filter [codegen id : 6] Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] Condition : isnotnull(ws_bill_customer_sk#61) @@ -298,30 +298,30 @@ Condition : isnotnull(ws_bill_customer_sk#61) Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#47] Right keys [1]: [ws_bill_customer_sk#61] Join type: Inner Join condition: None -(48) Project [codegen id : 10] +(48) Project [codegen id : 7] Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] Input [12]: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] (49) ReusedExchange [Reuses operator id: 84] Output [2]: [d_date_sk#65, d_year#66] -(50) BroadcastHashJoin [codegen id : 10] +(50) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#64] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(51) Project [codegen id : 10] +(51) Project [codegen id : 7] Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66] Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66] -(52) HashAggregate [codegen id : 10] +(52) HashAggregate [codegen id : 7] Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))] @@ -332,17 +332,17 @@ Results [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cus Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometColumnarToRow [codegen id : 11] +(54) CometNativeColumnarToRow Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] -(55) HashAggregate [codegen id : 11] +(55) HashAggregate [codegen id : 8] Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))#69] Results [2]: [c_customer_id#55 AS customer_id#70, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))#69,18,2) AS year_total#71] -(56) Filter [codegen id : 11] +(56) Filter [codegen id : 8] Input [2]: [customer_id#70, year_total#71] Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) @@ -350,13 +350,13 @@ Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) Input [2]: [customer_id#70, year_total#71] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 16] +(58) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#70] Join type: Inner Join condition: None -(59) Project [codegen id : 16] +(59) Project [codegen id : 12] Output [5]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, year_total#71] Input [6]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, customer_id#70, year_total#71] @@ -375,7 +375,7 @@ Condition : (isnotnull(c_customer_sk#72) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] Arguments: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60], [c_customer_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#73, 16, true, false, true) AS c_customer_id#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#74, 20, true, false, true) AS c_first_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#75, 30, true, false, true) AS c_last_name#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#76, 1, true, false, true) AS c_preferred_cust_flag#58, c_birth_country#77, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#78, 13, true, false, true) AS c_login#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#79, 50, true, false, true) AS c_email_address#60] -(63) CometColumnarToRow [codegen id : 14] +(63) CometNativeColumnarToRow Input [8]: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60] (64) Scan parquet spark_catalog.default.web_sales @@ -386,10 +386,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 12] +(65) ColumnarToRow [codegen id : 9] Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -(66) Filter [codegen id : 12] +(66) Filter [codegen id : 9] Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] Condition : isnotnull(ws_bill_customer_sk#80) @@ -397,30 +397,30 @@ Condition : isnotnull(ws_bill_customer_sk#80) Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(68) BroadcastHashJoin [codegen id : 14] +(68) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#72] Right keys [1]: [ws_bill_customer_sk#80] Join type: Inner Join condition: None -(69) Project [codegen id : 14] +(69) Project [codegen id : 10] Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] Input [12]: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] (70) ReusedExchange [Reuses operator id: 88] Output [2]: [d_date_sk#84, d_year#85] -(71) BroadcastHashJoin [codegen id : 14] +(71) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(72) Project [codegen id : 14] +(72) Project [codegen id : 10] Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#85] Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83, d_date_sk#84, d_year#85] -(73) HashAggregate [codegen id : 14] +(73) HashAggregate [codegen id : 10] Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#85] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))] @@ -431,10 +431,10 @@ Results [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cus Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(75) CometColumnarToRow [codegen id : 15] +(75) CometNativeColumnarToRow Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] -(76) HashAggregate [codegen id : 15] +(76) HashAggregate [codegen id : 11] Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))] @@ -445,13 +445,13 @@ Results [2]: [c_customer_id#55 AS customer_id#88, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#88, year_total#89] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(78) BroadcastHashJoin [codegen id : 16] +(78) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#88] Join type: Inner Join condition: (CASE WHEN (year_total#71 > 0.00) THEN (year_total#89 / year_total#71) END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#46 / year_total#26) END) -(79) Project [codegen id : 16] +(79) Project [codegen id : 12] Output [1]: [customer_preferred_cust_flag#45] Input [7]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, year_total#71, customer_id#88, year_total#89] @@ -463,7 +463,7 @@ Arguments: 100, [customer_preferred_cust_flag#45 ASC NULLS FIRST], [customer_pre Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (84) -+- * CometColumnarToRow (83) ++- CometNativeColumnarToRow (83) +- CometFilter (82) +- CometNativeScan parquet spark_catalog.default.date_dim (81) @@ -479,7 +479,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(83) CometColumnarToRow [codegen id : 1] +(83) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (84) BroadcastExchange @@ -488,7 +488,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 BroadcastExchange (88) -+- * CometColumnarToRow (87) ++- CometNativeColumnarToRow (87) +- CometFilter (86) +- CometNativeScan parquet spark_catalog.default.date_dim (85) @@ -504,7 +504,7 @@ ReadSchema: struct Input [2]: [d_date_sk#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(87) CometColumnarToRow [codegen id : 1] +(87) CometNativeColumnarToRow Input [2]: [d_date_sk#40, d_year#41] (88) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt index 0f9f19de77..47aa9517cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -24,23 +24,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -50,24 +50,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -77,19 +77,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -99,7 +99,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt index 6c9e276c01..80b29cda63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] @@ -8,17 +8,17 @@ TakeOrderedAndProject [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] @@ -31,69 +31,65 @@ TakeOrderedAndProject [customer_preferred_cust_flag] Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -103,25 +99,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt index 091ad62567..937c9228e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt @@ -420,7 +420,7 @@ Input [1]: [customer_preferred_cust_flag#42] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometFilter (75) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (77) BroadcastExchange @@ -445,7 +445,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometFilter (79) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) @@ -461,7 +461,7 @@ ReadSchema: struct Input [2]: [d_date_sk#38, d_year#39] Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#38, d_year#39] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt index 781eae9054..63858a8376 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -44,7 +44,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt index 4599ecf539..0cc605f69e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt @@ -24,11 +24,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -48,11 +46,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt index 091ad62567..937c9228e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt @@ -420,7 +420,7 @@ Input [1]: [customer_preferred_cust_flag#42] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometFilter (75) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (77) BroadcastExchange @@ -445,7 +445,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometFilter (79) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) @@ -461,7 +461,7 @@ ReadSchema: struct Input [2]: [d_date_sk#38, d_year#39] Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#38, d_year#39] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/extended.txt index 781eae9054..63858a8376 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -44,7 +44,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 4599ecf539..0cc605f69e 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 @@ -24,11 +24,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -48,11 +46,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/explain.txt index 19d16c378b..34d1d3917a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.web_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) @@ -54,37 +54,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] (21) Window Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt index 6c2a775097..15648408fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt index b7b0a89774..b580349644 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt index 93308784a9..0e486aa5dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt index fb83fd2f9a..c014d569f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt index 93308784a9..0e486aa5dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fb83fd2f9a..c014d569f4 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/explain.txt index dbacf525ef..cfff06dc67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * HashAggregate (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometColumnarExchange (34) +- * HashAggregate (33) +- * Project (32) @@ -17,22 +17,22 @@ : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) + : : : : +- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.store (4) : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometProject (12) : : : +- CometFilter (11) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) : : +- ReusedExchange (17) : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometProject (22) : +- CometFilter (21) : +- CometNativeScan parquet spark_catalog.default.customer_demographics (20) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.household_demographics (27) @@ -45,10 +45,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 1] Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 1] Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) @@ -63,20 +63,20 @@ ReadSchema: struct Input [1]: [s_store_sk#12] Condition : isnotnull(s_store_sk#12) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [s_store_sk#12] (7) BroadcastExchange Input [1]: [s_store_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 1] Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] @@ -95,33 +95,33 @@ Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) AS ca_state#16] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [2]: [ca_address_sk#13, ca_state#16] (14) BroadcastExchange Input [2]: [ca_address_sk#13, ca_state#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#3] Right keys [1]: [ca_address_sk#13] Join type: Inner Join condition: ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) -(16) Project [codegen id : 6] +(16) Project [codegen id : 1] Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] (17) ReusedExchange [Reuses operator id: 41] Output [1]: [d_date_sk#17] -(18) BroadcastHashJoin [codegen id : 6] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#10] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(19) Project [codegen id : 1] Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] @@ -140,20 +140,20 @@ Condition : (isnotnull(cd_demo_sk#18) AND ((((staticinvoke(class org.apache.spar Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] Arguments: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) AS cd_education_status#22] -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] (24) BroadcastExchange Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: ((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) -(26) Project [codegen id : 6] +(26) Project [codegen id : 1] Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22] Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] @@ -168,24 +168,24 @@ ReadSchema: struct Input [2]: [hd_demo_sk#23, hd_dep_count#24] Condition : (isnotnull(hd_demo_sk#23) AND ((hd_dep_count#24 = 3) OR (hd_dep_count#24 = 1))) -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [2]: [hd_demo_sk#23, hd_dep_count#24] (30) BroadcastExchange Input [2]: [hd_demo_sk#23, hd_dep_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#23] Join type: Inner Join condition: (((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#24 = 3)) OR (((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#24 = 1))) OR (((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#24 = 1))) -(32) Project [codegen id : 6] +(32) Project [codegen id : 1] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22, hd_demo_sk#23, hd_dep_count#24] -(33) HashAggregate [codegen id : 6] +(33) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Keys: [] Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] @@ -196,10 +196,10 @@ Results [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] -(36) HashAggregate [codegen id : 7] +(36) HashAggregate [codegen id : 2] Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] @@ -210,7 +210,7 @@ Results [4]: [avg(ss_quantity#5)#39 AS avg(ss_quantity)#43, cast((avg(UnscaledVa Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (41) -+- * CometColumnarToRow (40) ++- CometNativeColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometNativeScan parquet spark_catalog.default.date_dim (37) @@ -231,7 +231,7 @@ Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#17, d_year#47] Arguments: [d_date_sk#17], [d_date_sk#17] -(40) CometColumnarToRow [codegen id : 1] +(40) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (41) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt index 4c0d0b7a33..9ff51aab85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,31 +17,31 @@ HashAggregate : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt index a33ae5a161..d7dc065e83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) +WholeStageCodegen (2) HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (6) + WholeStageCodegen (1) HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] @@ -21,41 +21,31 @@ WholeStageCodegen (7) Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometNativeColumnarToRow + CometFilter [hd_demo_sk,hd_dep_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/explain.txt index 74da8ba883..d4881d2dff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/explain.txt @@ -200,7 +200,7 @@ Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesal Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -221,7 +221,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#17, d_year#18] Arguments: [d_date_sk#17], [d_date_sk#17] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt index 08e9beb692..5568227543 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt index 8ef882a435..3d2ac96c74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt index 74da8ba883..d4881d2dff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt @@ -200,7 +200,7 @@ Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesal Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -221,7 +221,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#17, d_year#18] Arguments: [d_date_sk#17], [d_date_sk#17] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/extended.txt index 08e9beb692..5568227543 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 8ef882a435..3d2ac96c74 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/explain.txt index f7377ccf8a..ad732bc935 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (110) +- * HashAggregate (109) - +- * CometColumnarToRow (108) + +- CometNativeColumnarToRow (108) +- CometColumnarExchange (107) +- * HashAggregate (106) +- * Expand (105) @@ -9,7 +9,7 @@ TakeOrderedAndProject (110) :- * Project (69) : +- * Filter (68) : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) + : +- CometNativeColumnarToRow (66) : +- CometColumnarExchange (65) : +- * HashAggregate (64) : +- * Project (63) @@ -23,12 +23,12 @@ TakeOrderedAndProject (110) : : : +- BroadcastExchange (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) + : : : :- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (48) : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) + : : : :- CometNativeColumnarToRow (36) : : : : +- CometHashAggregate (35) : : : : +- CometColumnarExchange (34) : : : : +- * HashAggregate (33) @@ -41,7 +41,7 @@ TakeOrderedAndProject (110) : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : +- BroadcastExchange (27) : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) + : : : : : :- CometNativeColumnarToRow (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- BroadcastExchange (25) @@ -53,7 +53,7 @@ TakeOrderedAndProject (110) : : : : : : : +- * ColumnarToRow (14) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometNativeColumnarToRow (18) : : : : : : +- CometFilter (17) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : +- ReusedExchange (22) @@ -70,7 +70,7 @@ TakeOrderedAndProject (110) : : : +- ReusedExchange (43) : : +- BroadcastExchange (58) : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) + : : :- CometNativeColumnarToRow (55) : : : +- CometFilter (54) : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : +- ReusedExchange (56) @@ -78,7 +78,7 @@ TakeOrderedAndProject (110) :- * Project (86) : +- * Filter (85) : +- * HashAggregate (84) - : +- * CometColumnarToRow (83) + : +- CometNativeColumnarToRow (83) : +- CometColumnarExchange (82) : +- * HashAggregate (81) : +- * Project (80) @@ -95,7 +95,7 @@ TakeOrderedAndProject (110) +- * Project (103) +- * Filter (102) +- * HashAggregate (101) - +- * CometColumnarToRow (100) + +- CometNativeColumnarToRow (100) +- CometColumnarExchange (99) +- * HashAggregate (98) +- * Project (97) @@ -119,10 +119,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -137,7 +137,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -166,7 +166,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -195,33 +195,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -326,7 +326,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -336,13 +336,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -350,7 +350,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -367,13 +367,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(69) Project [codegen id : 26] +(69) Project [codegen id : 15] Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#37 AS i_brand_id#54, i_class_id#38 AS i_class_id#55, i_category_id#39 AS i_category_id#56] Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] @@ -443,17 +443,17 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 51] +(71) ColumnarToRow [codegen id : 29] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -(72) Filter [codegen id : 51] +(72) Filter [codegen id : 29] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Condition : isnotnull(cs_item_sk#57) (73) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#61] -(74) BroadcastHashJoin [codegen id : 51] +(74) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#57] Right keys [1]: [ss_item_sk#61] Join type: LeftSemi @@ -462,30 +462,30 @@ Join condition: None (75) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(76) BroadcastHashJoin [codegen id : 51] +(76) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#57] Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(77) Project [codegen id : 51] +(77) Project [codegen id : 29] Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] (78) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#66] -(79) BroadcastHashJoin [codegen id : 51] +(79) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(80) Project [codegen id : 51] +(80) Project [codegen id : 29] Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] -(81) HashAggregate [codegen id : 51] +(81) HashAggregate [codegen id : 29] Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] @@ -496,21 +496,21 @@ Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71 Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(83) CometColumnarToRow [codegen id : 52] +(83) CometNativeColumnarToRow Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -(84) HashAggregate [codegen id : 52] +(84) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73, count(1)#74] Results [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73 AS sales#75, count(1)#74 AS number_sales#76] -(85) Filter [codegen id : 52] +(85) Filter [codegen id : 30] Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#75, number_sales#76] Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(86) Project [codegen id : 52] +(86) Project [codegen id : 30] Output [6]: [sales#75, number_sales#76, catalog AS channel#77, i_brand_id#63, i_class_id#64, i_category_id#65] Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#75, number_sales#76] @@ -522,17 +522,17 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(88) ColumnarToRow [codegen id : 77] +(88) ColumnarToRow [codegen id : 44] Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -(89) Filter [codegen id : 77] +(89) Filter [codegen id : 44] Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] Condition : isnotnull(ws_item_sk#78) (90) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#82] -(91) BroadcastHashJoin [codegen id : 77] +(91) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#78] Right keys [1]: [ss_item_sk#82] Join type: LeftSemi @@ -541,30 +541,30 @@ Join condition: None (92) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] -(93) BroadcastHashJoin [codegen id : 77] +(93) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#78] Right keys [1]: [i_item_sk#83] Join type: Inner Join condition: None -(94) Project [codegen id : 77] +(94) Project [codegen id : 44] Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86] Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] (95) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#87] -(96) BroadcastHashJoin [codegen id : 77] +(96) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#81] Right keys [1]: [d_date_sk#87] Join type: Inner Join condition: None -(97) Project [codegen id : 77] +(97) Project [codegen id : 44] Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86, d_date_sk#87] -(98) HashAggregate [codegen id : 77] +(98) HashAggregate [codegen id : 44] Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] @@ -575,31 +575,31 @@ Results [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92 Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] Arguments: hashpartitioning(i_brand_id#84, i_class_id#85, i_category_id#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(100) CometColumnarToRow [codegen id : 78] +(100) CometNativeColumnarToRow Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] -(101) HashAggregate [codegen id : 78] +(101) HashAggregate [codegen id : 45] Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94, count(1)#95] Results [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94 AS sales#96, count(1)#95 AS number_sales#97] -(102) Filter [codegen id : 78] +(102) Filter [codegen id : 45] Input [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sales#96, number_sales#97] Condition : (isnotnull(sales#96) AND (cast(sales#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(103) Project [codegen id : 78] +(103) Project [codegen id : 45] Output [6]: [sales#96, number_sales#97, web AS channel#98, i_brand_id#84, i_class_id#85, i_category_id#86] Input [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sales#96, number_sales#97] (104) Union -(105) Expand [codegen id : 79] +(105) Expand [codegen id : 46] Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] -(106) HashAggregate [codegen id : 79] +(106) HashAggregate [codegen id : 46] Input [7]: [sales#49, number_sales#50, channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] Keys [5]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] @@ -610,10 +610,10 @@ Results [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spa Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] Arguments: hashpartitioning(channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(108) CometColumnarToRow [codegen id : 80] +(108) CometNativeColumnarToRow Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] -(109) HashAggregate [codegen id : 80] +(109) HashAggregate [codegen id : 47] Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] Keys [5]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] Functions [2]: [sum(sales#49), sum(number_sales#50)] @@ -628,7 +628,7 @@ Arguments: 100, [channel#99 ASC NULLS FIRST, i_brand_id#100 ASC NULLS FIRST, i_c Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * HashAggregate (130) -+- * CometColumnarToRow (129) ++- CometNativeColumnarToRow (129) +- CometColumnarExchange (128) +- * HashAggregate (127) +- Union (126) @@ -656,19 +656,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#116), dynamicpruningexpression(ss_sold_date_sk#116 IN dynamicpruning#12)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 2] +(112) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116] (113) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#117] -(114) BroadcastHashJoin [codegen id : 2] +(114) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#116] Right keys [1]: [d_date_sk#117] Join type: Inner Join condition: None -(115) Project [codegen id : 2] +(115) Project [codegen id : 1] Output [2]: [ss_quantity#114 AS quantity#118, ss_list_price#115 AS list_price#119] Input [4]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116, d_date_sk#117] @@ -679,19 +679,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#122), dynamicpruningexpression(cs_sold_date_sk#122 IN dynamicpruning#12)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 4] +(117) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122] (118) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#123] -(119) BroadcastHashJoin [codegen id : 4] +(119) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#122] Right keys [1]: [d_date_sk#123] Join type: Inner Join condition: None -(120) Project [codegen id : 4] +(120) Project [codegen id : 2] Output [2]: [cs_quantity#120 AS quantity#124, cs_list_price#121 AS list_price#125] Input [4]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122, d_date_sk#123] @@ -702,25 +702,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_sold_date_sk#128 IN dynamicpruning#12)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 6] +(122) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128] (123) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#129] -(124) BroadcastHashJoin [codegen id : 6] +(124) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#128] Right keys [1]: [d_date_sk#129] Join type: Inner Join condition: None -(125) Project [codegen id : 6] +(125) Project [codegen id : 3] Output [2]: [ws_quantity#126 AS quantity#130, ws_list_price#127 AS list_price#131] Input [4]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128, d_date_sk#129] (126) Union -(127) HashAggregate [codegen id : 7] +(127) HashAggregate [codegen id : 4] Input [2]: [quantity#118, list_price#119] Keys: [] Functions [1]: [partial_avg((cast(quantity#118 as decimal(10,0)) * list_price#119))] @@ -731,10 +731,10 @@ Results [2]: [sum#134, count#135] Input [2]: [sum#134, count#135] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(129) CometColumnarToRow [codegen id : 8] +(129) CometNativeColumnarToRow Input [2]: [sum#134, count#135] -(130) HashAggregate [codegen id : 8] +(130) HashAggregate [codegen id : 5] Input [2]: [sum#134, count#135] Keys: [] Functions [1]: [avg((cast(quantity#118 as decimal(10,0)) * list_price#119))] @@ -749,7 +749,7 @@ Subquery:4 Hosting operator id = 121 Hosting Expression = ws_sold_date_sk#128 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (135) -+- * CometColumnarToRow (134) ++- CometNativeColumnarToRow (134) +- CometProject (133) +- CometFilter (132) +- CometNativeScan parquet spark_catalog.default.date_dim (131) @@ -770,7 +770,7 @@ Condition : ((((isnotnull(d_year#138) AND isnotnull(d_moy#139)) AND (d_year#138 Input [3]: [d_date_sk#40, d_year#138, d_moy#139] Arguments: [d_date_sk#40], [d_date_sk#40] -(134) CometColumnarToRow [codegen id : 1] +(134) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (135) BroadcastExchange @@ -779,7 +779,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (140) -+- * CometColumnarToRow (139) ++- CometNativeColumnarToRow (139) +- CometProject (138) +- CometFilter (137) +- CometNativeScan parquet spark_catalog.default.date_dim (136) @@ -800,7 +800,7 @@ Condition : (((isnotnull(d_year#140) AND (d_year#140 >= 1999)) AND (d_year#140 < Input [2]: [d_date_sk#24, d_year#140] Arguments: [d_date_sk#24], [d_date_sk#24] -(139) CometColumnarToRow [codegen id : 1] +(139) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (140) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt index dfd3434d90..223593e3d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -9,7 +9,7 @@ TakeOrderedAndProject : +- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,12 +39,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -57,19 +57,19 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -82,13 +82,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -101,16 +101,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -124,28 +124,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -158,13 +158,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -177,16 +177,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -200,16 +200,16 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -217,7 +217,7 @@ TakeOrderedAndProject : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -232,12 +232,12 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -250,13 +250,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -269,16 +269,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -292,28 +292,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -326,13 +326,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -345,16 +345,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -368,16 +368,16 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -385,7 +385,7 @@ TakeOrderedAndProject +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -400,12 +400,12 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -418,13 +418,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -437,16 +437,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -460,28 +460,28 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -494,13 +494,13 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -513,16 +513,16 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -536,16 +536,16 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt index c989fe9a81..f348396ac0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt @@ -1,28 +1,28 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) + WholeStageCodegen (47) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) + WholeStageCodegen (46) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (26) + WholeStageCodegen (15) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #3 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #13 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -31,7 +31,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,10 +50,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [d_date_sk] #7 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -66,30 +66,28 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (11) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (10) + WholeStageCodegen (5) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [brand_id,class_id,category_id] CometColumnarExchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -101,23 +99,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) + WholeStageCodegen (2) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (1) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -129,18 +125,16 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 - WholeStageCodegen (9) + WholeStageCodegen (4) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -156,25 +150,25 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #12 - WholeStageCodegen (23) + WholeStageCodegen (13) BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) + WholeStageCodegen (30) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) + WholeStageCodegen (29) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -192,15 +186,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) + WholeStageCodegen (45) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt index 22f1896b63..b77116db1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * CometColumnarToRow (128) ++- CometNativeColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 < Input [2]: [d_date_sk#26, d_year#125] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) CometColumnarToRow [codegen id : 1] +(128) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (129) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt index 4af04a7846..49ad7403ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -213,7 +213,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -277,7 +277,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -355,7 +355,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -419,7 +419,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt index 79c782f2ca..fea1950918 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt @@ -44,12 +44,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #4 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -68,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt index 22f1896b63..b77116db1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * CometColumnarToRow (128) ++- CometNativeColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 < Input [2]: [d_date_sk#26, d_year#125] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) CometColumnarToRow [codegen id : 1] +(128) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (129) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/extended.txt index 4af04a7846..49ad7403ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -213,7 +213,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -277,7 +277,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -355,7 +355,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -419,7 +419,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 79c782f2ca..fea1950918 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 @@ -44,12 +44,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #4 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -68,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt index 563fa0a20b..6afb8eabee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (87) +- * BroadcastHashJoin Inner BuildRight (86) :- * Filter (68) : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) + : +- CometNativeColumnarToRow (66) : +- CometColumnarExchange (65) : +- * HashAggregate (64) : +- * Project (63) @@ -17,12 +17,12 @@ TakeOrderedAndProject (87) : : : +- BroadcastExchange (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) + : : : :- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (48) : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) + : : : :- CometNativeColumnarToRow (36) : : : : +- CometHashAggregate (35) : : : : +- CometColumnarExchange (34) : : : : +- * HashAggregate (33) @@ -35,7 +35,7 @@ TakeOrderedAndProject (87) : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : +- BroadcastExchange (27) : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) + : : : : : :- CometNativeColumnarToRow (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- BroadcastExchange (25) @@ -47,7 +47,7 @@ TakeOrderedAndProject (87) : : : : : : : +- * ColumnarToRow (14) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometNativeColumnarToRow (18) : : : : : : +- CometFilter (17) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : +- ReusedExchange (22) @@ -64,7 +64,7 @@ TakeOrderedAndProject (87) : : : +- ReusedExchange (43) : : +- BroadcastExchange (58) : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) + : : :- CometNativeColumnarToRow (55) : : : +- CometFilter (54) : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : +- ReusedExchange (56) @@ -72,7 +72,7 @@ TakeOrderedAndProject (87) +- BroadcastExchange (85) +- * Filter (84) +- * HashAggregate (83) - +- * CometColumnarToRow (82) + +- CometNativeColumnarToRow (82) +- CometColumnarExchange (81) +- * HashAggregate (80) +- * Project (79) @@ -96,10 +96,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -114,7 +114,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -143,7 +143,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -172,33 +172,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -303,7 +303,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -313,13 +313,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -327,7 +327,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -344,13 +344,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -416,17 +416,17 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 50] +(70) ColumnarToRow [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(71) Filter [codegen id : 50] +(71) Filter [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) (72) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#59] -(73) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#59] Join type: LeftSemi @@ -435,30 +435,30 @@ Join condition: None (74) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(75) BroadcastHashJoin [codegen id : 50] +(75) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#60] Join type: Inner Join condition: None -(76) Project [codegen id : 50] +(76) Project [codegen id : 28] Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] (77) ReusedExchange [Reuses operator id: 126] Output [1]: [d_date_sk#64] -(78) BroadcastHashJoin [codegen id : 50] +(78) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(79) Project [codegen id : 50] +(79) Project [codegen id : 28] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -(80) HashAggregate [codegen id : 50] +(80) HashAggregate [codegen id : 28] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] @@ -469,17 +469,17 @@ Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(82) CometColumnarToRow [codegen id : 51] +(82) CometNativeColumnarToRow Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -(83) HashAggregate [codegen id : 51] +(83) HashAggregate [codegen id : 29] Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71, count(1)#72] Results [6]: [store AS channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71 AS sales#74, count(1)#72 AS number_sales#75] -(84) Filter [codegen id : 51] +(84) Filter [codegen id : 29] Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -487,7 +487,7 @@ Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(Reu Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 52] +(86) BroadcastHashJoin [codegen id : 30] Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Join type: Inner @@ -501,7 +501,7 @@ Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * HashAggregate (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometColumnarExchange (105) +- * HashAggregate (104) +- Union (103) @@ -529,19 +529,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#12)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 2] +(89) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] (90) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#79] -(91) BroadcastHashJoin [codegen id : 2] +(91) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#78] Right keys [1]: [d_date_sk#79] Join type: Inner Join condition: None -(92) Project [codegen id : 2] +(92) Project [codegen id : 1] Output [2]: [ss_quantity#76 AS quantity#80, ss_list_price#77 AS list_price#81] Input [4]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#79] @@ -552,19 +552,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#12)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 4] +(94) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] (95) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#85] -(96) BroadcastHashJoin [codegen id : 4] +(96) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#84] Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(97) Project [codegen id : 4] +(97) Project [codegen id : 2] Output [2]: [cs_quantity#82 AS quantity#86, cs_list_price#83 AS list_price#87] Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#85] @@ -575,25 +575,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#12)] ReadSchema: struct -(99) ColumnarToRow [codegen id : 6] +(99) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] (100) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#91] -(101) BroadcastHashJoin [codegen id : 6] +(101) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#90] Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(102) Project [codegen id : 6] +(102) Project [codegen id : 3] Output [2]: [ws_quantity#88 AS quantity#92, ws_list_price#89 AS list_price#93] Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#91] (103) Union -(104) HashAggregate [codegen id : 7] +(104) HashAggregate [codegen id : 4] Input [2]: [quantity#80, list_price#81] Keys: [] Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] @@ -604,10 +604,10 @@ Results [2]: [sum#96, count#97] Input [2]: [sum#96, count#97] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(106) CometColumnarToRow [codegen id : 8] +(106) CometNativeColumnarToRow Input [2]: [sum#96, count#97] -(107) HashAggregate [codegen id : 8] +(107) HashAggregate [codegen id : 5] Input [2]: [sum#96, count#97] Keys: [] Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] @@ -622,7 +622,7 @@ Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (112) -+- * CometColumnarToRow (111) ++- CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometNativeScan parquet spark_catalog.default.date_dim (108) @@ -643,7 +643,7 @@ Condition : ((isnotnull(d_week_seq#100) AND (d_week_seq#100 = Subquery scalar-su Input [2]: [d_date_sk#40, d_week_seq#100] Arguments: [d_date_sk#40], [d_date_sk#40] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (112) BroadcastExchange @@ -651,7 +651,7 @@ Input [1]: [d_date_sk#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] Subquery:6 Hosting operator id = 109 Hosting Expression = Subquery scalar-subquery#101, [id=#102] -* CometColumnarToRow (116) +CometNativeColumnarToRow (116) +- CometProject (115) +- CometFilter (114) +- CometNativeScan parquet spark_catalog.default.date_dim (113) @@ -672,12 +672,12 @@ Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] Arguments: [d_week_seq#103], [d_week_seq#103] -(116) CometColumnarToRow [codegen id : 1] +(116) CometNativeColumnarToRow Input [1]: [d_week_seq#103] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometNativeScan parquet spark_catalog.default.date_dim (117) @@ -698,7 +698,7 @@ Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1999)) AND (d_year#107 < Input [2]: [d_date_sk#24, d_year#107] Arguments: [d_date_sk#24], [d_date_sk#24] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (121) BroadcastExchange @@ -713,7 +713,7 @@ Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquer Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 BroadcastExchange (126) -+- * CometColumnarToRow (125) ++- CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometNativeScan parquet spark_catalog.default.date_dim (122) @@ -734,7 +734,7 @@ Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = Subquery scalar-su Input [2]: [d_date_sk#64, d_week_seq#108] Arguments: [d_date_sk#64], [d_date_sk#64] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_date_sk#64] (126) BroadcastExchange @@ -742,7 +742,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:12 Hosting operator id = 123 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* CometColumnarToRow (130) +CometNativeColumnarToRow (130) +- CometProject (129) +- CometFilter (128) +- CometNativeScan parquet spark_catalog.default.date_dim (127) @@ -763,7 +763,7 @@ Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] Arguments: [d_week_seq#111], [d_week_seq#111] -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [1]: [d_week_seq#111] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt index c4138c254a..7921f2fa61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject :- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -23,7 +23,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -33,12 +33,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -51,11 +51,11 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,12 +63,12 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -81,13 +81,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -100,16 +100,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,28 +123,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -157,13 +157,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -176,16 +176,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -199,20 +199,20 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -221,7 +221,7 @@ TakeOrderedAndProject +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -234,11 +234,11 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -246,12 +246,12 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -264,13 +264,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -283,16 +283,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -306,28 +306,28 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -340,13 +340,13 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -359,16 +359,16 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -382,20 +382,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt index b0eae963c3..45d526a696 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (30) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #12 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -21,7 +21,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #6 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -56,37 +56,33 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (5) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [brand_id,class_id,category_id] CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -98,23 +94,21 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 - WholeStageCodegen (4) + WholeStageCodegen (2) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #8 - WholeStageCodegen (3) + WholeStageCodegen (1) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -126,18 +120,16 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 - WholeStageCodegen (9) + WholeStageCodegen (4) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -153,10 +145,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 - WholeStageCodegen (23) + WholeStageCodegen (13) BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter @@ -165,14 +157,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (51) + WholeStageCodegen (29) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + WholeStageCodegen (28) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -185,19 +177,15 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt index 879213d892..845b3a9e58 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#99] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subq Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#104] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt index f56d229b68..c1b791f357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt @@ -44,11 +44,11 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,11 +197,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -224,7 +224,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,7 +288,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/simplified.txt index fb9abae378..71f9d72e7d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/simplified.txt @@ -39,19 +39,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -131,19 +125,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt index 879213d892..845b3a9e58 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#99] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subq Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#104] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/extended.txt index f56d229b68..c1b791f357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/extended.txt @@ -44,11 +44,11 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,11 +197,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -224,7 +224,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,7 +288,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fb9abae378..71f9d72e7d 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 @@ -39,19 +39,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -131,19 +125,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/explain.txt index 594939616e..a9b46e19b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -14,11 +14,11 @@ TakeOrderedAndProject (24) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) + : : +- CometNativeColumnarToRow (6) : : +- CometFilter (5) : : +- CometNativeScan parquet spark_catalog.default.customer (4) : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) + : +- CometNativeColumnarToRow (13) : +- CometProject (12) : +- CometFilter (11) : +- CometNativeScan parquet spark_catalog.default.customer_address (10) @@ -33,10 +33,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_bill_customer_sk#1) @@ -51,20 +51,20 @@ ReadSchema: struct Input [2]: [c_customer_sk#5, c_current_addr_sk#6] Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [2]: [c_customer_sk#5, c_current_addr_sk#6] (7) BroadcastExchange Input [2]: [c_customer_sk#5, c_current_addr_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#5] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] @@ -83,37 +83,37 @@ Condition : isnotnull(ca_address_sk#7) Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#9, 10, true, false, true) AS ca_zip#11] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] (14) BroadcastExchange Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#6] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) -(16) Project [codegen id : 4] +(16) Project [codegen id : 1] Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] (17) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#12] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [2]: [cs_sales_price#2, ca_zip#11] Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [2]: [cs_sales_price#2, ca_zip#11] Keys [1]: [ca_zip#11] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] @@ -124,10 +124,10 @@ Results [2]: [ca_zip#11, sum#14] Input [2]: [ca_zip#11, sum#14] Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [2]: [ca_zip#11, sum#14] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [2]: [ca_zip#11, sum#14] Keys [1]: [ca_zip#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] @@ -142,7 +142,7 @@ Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : ((((isnotnull(d_qoy#18) AND isnotnull(d_year#17)) AND (d_qoy#18 = 2) Input [3]: [d_date_sk#12, d_year#17, d_qoy#18] Arguments: [d_date_sk#12], [d_date_sk#12] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#12] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt index 8ed98bc5cb..d0d170db53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -14,21 +14,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/simplified.txt index f31442dcfe..0c2174ad74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_zip] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [ca_zip,cs_sales_price] [sum,sum] Project [cs_sales_price,ca_zip] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -18,26 +18,20 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/explain.txt index af9b2efbd1..d7590ade8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/explain.txt @@ -142,7 +142,7 @@ Input [2]: [ca_zip#11, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2) Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#12] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt index 6de0c64850..55382845fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/simplified.txt index c39b96efe3..97974aec5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/simplified.txt @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt index af9b2efbd1..d7590ade8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt @@ -142,7 +142,7 @@ Input [2]: [ca_zip#11, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2) Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#12] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/extended.txt index 6de0c64850..55382845fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 c39b96efe3..97974aec5d 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 @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/explain.txt index 2a3c8932ec..49499b4b09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (41) +CometNativeColumnarToRow (41) +- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -204,7 +204,7 @@ Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [3]: [cs_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -230,6 +230,6 @@ Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -(41) CometColumnarToRow [codegen id : 2] +(41) CometNativeColumnarToRow Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/extended.txt index 9889eeaca3..cb53684dcf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/extended.txt @@ -1,9 +1,9 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/simplified.txt index 429d83d08c..3254fa6c33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/simplified.txt @@ -1,45 +1,43 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_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] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_warehouse_sk] - CometSort [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometExchange [cs_order_number] #2 - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] +CometNativeColumnarToRow + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + 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] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] + CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] + CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_warehouse_sk] + CometSort [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometExchange [cs_order_number] #2 + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + CometSort [cs_warehouse_sk,cs_order_number] + CometExchange [cs_order_number] #3 + CometProject [cs_warehouse_sk,cs_order_number] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + CometSort [cr_order_number] + CometExchange [cr_order_number] #4 + CometProject [cr_order_number] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cc_call_center_sk] #7 + CometProject [cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/explain.txt index 5005b676fc..f4f0b9e155 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- ReusedExchange (19) : : +- ReusedExchange (22) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.store (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.item (32) @@ -52,10 +52,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) @@ -78,13 +78,13 @@ Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnot Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 8] +(8) BroadcastHashJoin [codegen id : 3] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] Join type: Inner Join condition: None -(9) Project [codegen id : 8] +(9) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] @@ -107,52 +107,52 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 3] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] Join type: Inner Join condition: None -(15) Project [codegen id : 8] +(15) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] (16) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#18] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(18) Project [codegen id : 3] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#18] (19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#19] -(20) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_returned_date_sk#12] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(21) Project [codegen id : 8] +(21) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] (22) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(24) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] @@ -171,20 +171,20 @@ Condition : isnotnull(s_store_sk#21) Input [2]: [s_store_sk#21, s_state#22] Arguments: [s_store_sk#21, s_state#23], [s_store_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#22, 2, true, false, true) AS s_state#23] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [2]: [s_store_sk#21, s_state#23] (29) BroadcastExchange Input [2]: [s_store_sk#21, s_state#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#21, s_state#23] @@ -203,24 +203,24 @@ Condition : isnotnull(i_item_sk#24) Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] Arguments: [i_item_sk#24, i_item_id#27, i_item_desc#26], [i_item_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#25, 16, true, false, true) AS i_item_id#27, i_item_desc#26] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] (36) BroadcastExchange Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#24] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(38) Project [codegen id : 3] Output [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_sk#24, i_item_id#27, i_item_desc#26] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 3] Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] @@ -231,10 +231,10 @@ Results [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] Arguments: hashpartitioning(i_item_id#27, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 4] Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] @@ -249,7 +249,7 @@ Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [d_date_sk#18, d_quarter_name#85] Arguments: [d_date_sk#18], [d_date_sk#18] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#18] (48) BroadcastExchange @@ -279,7 +279,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -300,7 +300,7 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [d_date_sk#19, d_quarter_name#86] Arguments: [d_date_sk#19], [d_date_sk#19] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt index 26e924fd7e..15c6bec5dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,27 +42,27 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/simplified.txt index c63dd716a1..591d904435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,12 +39,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -63,17 +59,13 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk,i_item_id,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/explain.txt index f502a5cd8d..0ec04f50e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/explain.txt @@ -256,7 +256,7 @@ Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#21] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/simplified.txt index 1d48f96b09..a274ade9c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/simplified.txt @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt index f502a5cd8d..0ec04f50e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt @@ -256,7 +256,7 @@ Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#21] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 1d48f96b09..a274ade9c5 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 @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/explain.txt index 0503116eb3..8c843bd4f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (46) +- * HashAggregate (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometColumnarExchange (43) +- * HashAggregate (42) +- * Expand (41) @@ -21,27 +21,27 @@ TakeOrderedAndProject (46) : : : : : : +- * ColumnarToRow (2) : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : +- BroadcastExchange (8) - : : : : : +- * CometColumnarToRow (7) + : : : : : +- CometNativeColumnarToRow (7) : : : : : +- CometProject (6) : : : : : +- CometFilter (5) : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : : : +- BroadcastExchange (15) - : : : : +- * CometColumnarToRow (14) + : : : : +- CometNativeColumnarToRow (14) : : : : +- CometProject (13) : : : : +- CometFilter (12) : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) : : : +- BroadcastExchange (21) - : : : +- * CometColumnarToRow (20) + : : : +- CometNativeColumnarToRow (20) : : : +- CometFilter (19) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) : : +- BroadcastExchange (28) - : : +- * CometColumnarToRow (27) + : : +- CometNativeColumnarToRow (27) : : +- CometProject (26) : : +- CometFilter (25) : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) : +- ReusedExchange (31) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.item (34) @@ -55,10 +55,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) @@ -77,20 +77,20 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [cd_demo_sk#11, cd_dep_count#14] (8) BroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 1] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] @@ -109,20 +109,20 @@ Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15 Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) CometColumnarToRow [codegen id : 2] +(14) CometNativeColumnarToRow Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] (15) BroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 7] +(16) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(17) Project [codegen id : 7] +(17) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] @@ -137,20 +137,20 @@ ReadSchema: struct Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [1]: [cd_demo_sk#20] (21) BroadcastExchange Input [1]: [cd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 1] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -169,33 +169,33 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] -(27) CometColumnarToRow [codegen id : 4] +(27) CometNativeColumnarToRow Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (28) BroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(30) Project [codegen id : 7] +(30) Project [codegen id : 1] Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (31) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#26] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] @@ -214,28 +214,28 @@ Condition : isnotnull(i_item_sk#27) Input [2]: [i_item_sk#27, i_item_id#28] Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] -(37) CometColumnarToRow [codegen id : 6] +(37) CometNativeColumnarToRow Input [2]: [i_item_sk#27, i_item_id#29] (38) BroadcastExchange Input [2]: [i_item_sk#27, i_item_id#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 7] +(39) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(40) Project [codegen id : 7] +(40) Project [codegen id : 1] Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] -(41) Expand [codegen id : 7] +(41) Expand [codegen id : 1] Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -(42) HashAggregate [codegen id : 7] +(42) HashAggregate [codegen id : 1] Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] @@ -246,10 +246,10 @@ Results [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_gro Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(44) CometColumnarToRow [codegen id : 8] +(44) CometNativeColumnarToRow Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -(45) HashAggregate [codegen id : 8] +(45) HashAggregate [codegen id : 2] Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] @@ -264,7 +264,7 @@ Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (51) -+- * CometColumnarToRow (50) ++- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometNativeScan parquet spark_catalog.default.date_dim (47) @@ -285,7 +285,7 @@ Condition : ((isnotnull(d_year#77) AND (d_year#77 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#77] Arguments: [d_date_sk#26], [d_date_sk#26] -(50) CometColumnarToRow [codegen id : 1] +(50) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt index b47fce49b3..53fa7b6342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -21,36 +21,36 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/simplified.txt index 77a45c46cd..0162997d1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] @@ -25,50 +25,38 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeColumnarToRow + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeColumnarToRow + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometNativeColumnarToRow + CometFilter [cd_demo_sk] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/explain.txt index 5fefd21bfa..87a8904c51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/explain.txt @@ -249,7 +249,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, ag Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt index 0b554c7e7c..7958390823 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/simplified.txt index 6c2b8b2e4f..6abfbb11b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt index 5fefd21bfa..87a8904c51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt @@ -249,7 +249,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, ag Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/extended.txt index 0b554c7e7c..7958390823 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6c2b8b2e4f..6abfbb11b2 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 @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/explain.txt index 539af26bc5..6465f46bc9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (36) +CometNativeColumnarToRow (36) +- CometTakeOrderedAndProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -203,6 +203,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt index fbe6798e3d..fa37da7c94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/simplified.txt index 675500cd0b..f6eefbf9a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/simplified.txt @@ -1,38 +1,36 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_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 [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] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] - CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] - CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_zip] +CometNativeColumnarToRow + CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_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 [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] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] + CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 + CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_zip] #5 + CometProject [ca_zip] [ca_address_sk,ca_zip] + CometFilter [ca_address_sk,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometProject [s_zip] [s_store_sk,s_zip] + CometFilter [s_store_sk,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/explain.txt index 40e213818e..9e359c1199 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (34) +CometNativeColumnarToRow (34) +- CometSort (33) +- CometExchange (32) +- CometProject (31) @@ -188,6 +188,6 @@ Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREM 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: [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 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow 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.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/extended.txt index 7e6f4d3a3c..c3d8dd11af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/simplified.txt index e4b6e81639..6d51b41514 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/simplified.txt @@ -1,36 +1,34 @@ -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)] - 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] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - 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_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] - CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [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] - CometFilter [d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] +CometNativeColumnarToRow + 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)] + 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] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + 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_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] + CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [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] + CometFilter [d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/explain.txt index c699bf18de..e0c7fc73ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.catalog_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) @@ -54,37 +54,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] (21) Window Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt index 369ec68bb4..6634edcbed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/simplified.txt index fb0ed62abe..21e87e03b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/explain.txt index c23383e09a..231e3847e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/simplified.txt index 2958d060fe..b3a099fb40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt index c23383e09a..231e3847e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 2958d060fe..b3a099fb40 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/explain.txt index 65cbe8c435..69c85a6649 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (25) +- * Filter (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -15,11 +15,11 @@ TakeOrderedAndProject (25) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.inventory (1) : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) + : : +- CometNativeColumnarToRow (6) : : +- CometFilter (5) : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) + : +- CometNativeColumnarToRow (13) : +- CometProject (12) : +- CometFilter (11) : +- CometNativeScan parquet spark_catalog.default.item (10) @@ -34,10 +34,10 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) @@ -52,20 +52,20 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Condition : isnotnull(w_warehouse_sk#6) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] (7) BroadcastExchange Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#6] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] @@ -84,37 +84,37 @@ Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) A Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#9, 16, true, false, true) AS i_item_id#11] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [2]: [i_item_sk#8, i_item_id#11] (14) BroadcastExchange Input [2]: [i_item_sk#8, i_item_id#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(16) Project [codegen id : 4] +(16) Project [codegen id : 1] Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] (17) ReusedExchange [Reuses operator id: 29] Output [2]: [d_date_sk#12, d_date#13] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] Keys [2]: [w_warehouse_name#7, i_item_id#11] Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] @@ -125,17 +125,17 @@ Results [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] Keys [2]: [w_warehouse_name#7, i_item_id#11] Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] Aggregate Attributes [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19] Results [4]: [w_warehouse_name#7, i_item_id#11, sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_before#20, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19 AS inv_after#21] -(24) Filter [codegen id : 5] +(24) Filter [codegen id : 2] Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] Condition : (CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) <= 1.5) END) @@ -147,7 +147,7 @@ Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRS Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_date#13] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt index 51fcfd010a..407648ea84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,20 +15,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/simplified.txt index 94925f8911..7c6d29a502 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) + WholeStageCodegen (2) Filter [inv_before,inv_after] HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] BroadcastHashJoin [inv_date_sk,d_date_sk] @@ -19,25 +19,19 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/explain.txt index 5656916223..1053830b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/explain.txt @@ -142,7 +142,7 @@ Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_date#13] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt index ef02334507..097f5b20cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/simplified.txt index 1c2e80c991..9fc6fdc42b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/simplified.txt @@ -16,11 +16,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 CometFilter [w_warehouse_sk,w_warehouse_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt index 5656916223..1053830b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt @@ -142,7 +142,7 @@ Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_date#13] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/extended.txt index ef02334507..097f5b20cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 1c2e80c991..9fc6fdc42b 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 @@ -16,11 +16,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 CometFilter [w_warehouse_sk,w_warehouse_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/explain.txt index f25f305679..bb531620cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (25) +- * HashAggregate (24) - +- * CometColumnarToRow (23) + +- CometNativeColumnarToRow (23) +- CometColumnarExchange (22) +- * HashAggregate (21) +- * Expand (20) @@ -16,12 +16,12 @@ TakeOrderedAndProject (25) : : : +- Scan parquet spark_catalog.default.inventory (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.warehouse (14) @@ -34,23 +34,23 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (4) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] @@ -69,20 +69,20 @@ Condition : isnotnull(i_item_sk#7) Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] (11) BroadcastExchange Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] @@ -97,28 +97,28 @@ ReadSchema: struct Input [1]: [w_warehouse_sk#16] Condition : isnotnull(w_warehouse_sk#16) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [w_warehouse_sk#16] (17) BroadcastExchange Input [1]: [w_warehouse_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#16] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] -(20) Expand [codegen id : 4] +(20) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] Arguments: [[inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#3, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] @@ -129,10 +129,10 @@ Results [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_gr Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] Arguments: hashpartitioning(i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 5] +(23) CometNativeColumnarToRow Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] Functions [1]: [avg(inv_quantity_on_hand#3)] @@ -147,7 +147,7 @@ Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.date_dim (26) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_mo Input [2]: [d_date_sk#6, d_month_seq#28] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt index 9b15a52cff..e1766e64d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -15,22 +15,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/simplified.txt index 092e187177..baff9284b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,26 +19,20 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometNativeColumnarToRow + CometFilter [w_warehouse_sk] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/explain.txt index 02cfd90b87..0486c4e213 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/explain.txt @@ -147,7 +147,7 @@ Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt index 7369619d2f..34e673af7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/simplified.txt index 9119ee749d..2cc520ff12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/simplified.txt @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt index 02cfd90b87..0486c4e213 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt @@ -147,7 +147,7 @@ Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/extended.txt index 7369619d2f..34e673af7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 9119ee749d..2cc520ff12 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 @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/explain.txt index 029c7fd3df..150428e457 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (69) +CometNativeColumnarToRow (69) +- CometHashAggregate (68) +- CometExchange (67) +- CometHashAggregate (66) @@ -18,7 +18,7 @@ : : : +- * Project (20) : : : +- * Filter (19) : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) + : : : +- CometNativeColumnarToRow (17) : : : +- CometColumnarExchange (16) : : : +- * HashAggregate (15) : : : +- * Project (14) @@ -30,7 +30,7 @@ : : : : : +- Scan parquet spark_catalog.default.store_sales (3) : : : : +- ReusedExchange (6) : : : +- BroadcastExchange (12) - : : : +- * CometColumnarToRow (11) + : : : +- CometNativeColumnarToRow (11) : : : +- CometFilter (10) : : : +- CometNativeScan parquet spark_catalog.default.item (9) : : +- CometSort (39) @@ -77,7 +77,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (3) Scan parquet spark_catalog.default.store_sales @@ -88,23 +88,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(4) ColumnarToRow [codegen id : 3] +(4) ColumnarToRow [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(5) Filter [codegen id : 3] +(5) Filter [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) (6) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#10, d_date#11] -(7) BroadcastHashJoin [codegen id : 3] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(8) Project [codegen id : 3] +(8) Project [codegen id : 1] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] @@ -119,24 +119,24 @@ ReadSchema: struct Input [2]: [i_item_sk#12, i_item_desc#13] Condition : isnotnull(i_item_sk#12) -(11) CometColumnarToRow [codegen id : 2] +(11) CometNativeColumnarToRow Input [2]: [i_item_sk#12, i_item_desc#13] (12) BroadcastExchange Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 1] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 1] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] @@ -147,21 +147,21 @@ Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(19) Filter [codegen id : 4] +(19) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(20) Project [codegen id : 4] +(20) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] @@ -169,13 +169,13 @@ Input [2]: [item_sk#18, cnt#19] Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 5] +(22) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(23) Project [codegen id : 5] +(23) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -296,19 +296,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#6)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 10] +(49) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] (50) ReusedExchange [Reuses operator id: 21] Output [1]: [item_sk#39] -(51) BroadcastHashJoin [codegen id : 10] +(51) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#34] Right keys [1]: [item_sk#39] Join type: LeftSemi Join condition: None -(52) Project [codegen id : 10] +(52) Project [codegen id : 6] Output [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] @@ -379,14 +379,14 @@ Input [2]: [sum#48, isEmpty#49] Keys: [] Functions [1]: [sum(sales#33)] -(69) CometColumnarToRow [codegen id : 11] +(69) CometNativeColumnarToRow Input [1]: [sum(sales)#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometProject (72) +- CometFilter (71) +- CometNativeScan parquet spark_catalog.default.date_dim (70) @@ -407,7 +407,7 @@ Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2 Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Arguments: [d_date_sk#30], [d_date_sk#30] -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [1]: [d_date_sk#30] (74) BroadcastExchange @@ -416,7 +416,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometFilter (76) +- CometNativeScan parquet spark_catalog.default.date_dim (75) @@ -437,7 +437,7 @@ Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#51] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (79) BroadcastExchange @@ -446,11 +446,11 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#28, [id=#29] * HashAggregate (99) -+- * CometColumnarToRow (98) ++- CometNativeColumnarToRow (98) +- CometColumnarExchange (97) +- * HashAggregate (96) +- * HashAggregate (95) - +- * CometColumnarToRow (94) + +- CometNativeColumnarToRow (94) +- CometColumnarExchange (93) +- * HashAggregate (92) +- * Project (91) @@ -461,7 +461,7 @@ Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquer : : +- * ColumnarToRow (81) : : +- Scan parquet spark_catalog.default.store_sales (80) : +- BroadcastExchange (86) - : +- * CometColumnarToRow (85) + : +- CometNativeColumnarToRow (85) : +- CometFilter (84) : +- CometNativeScan parquet spark_catalog.default.customer (83) +- ReusedExchange (89) @@ -475,10 +475,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 3] +(81) ColumnarToRow [codegen id : 1] Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -(82) Filter [codegen id : 3] +(82) Filter [codegen id : 1] Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] Condition : isnotnull(ss_customer_sk#52) @@ -493,37 +493,37 @@ ReadSchema: struct Input [1]: [c_customer_sk#57] Condition : isnotnull(c_customer_sk#57) -(85) CometColumnarToRow [codegen id : 1] +(85) CometNativeColumnarToRow Input [1]: [c_customer_sk#57] (86) BroadcastExchange Input [1]: [c_customer_sk#57] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(87) BroadcastHashJoin [codegen id : 3] +(87) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#52] Right keys [1]: [c_customer_sk#57] Join type: Inner Join condition: None -(88) Project [codegen id : 3] +(88) Project [codegen id : 1] Output [4]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] Input [5]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] (89) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#58] -(90) BroadcastHashJoin [codegen id : 3] +(90) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#55] Right keys [1]: [d_date_sk#58] Join type: Inner Join condition: None -(91) Project [codegen id : 3] +(91) Project [codegen id : 1] Output [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] Input [5]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57, d_date_sk#58] -(92) HashAggregate [codegen id : 3] +(92) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] Keys [1]: [c_customer_sk#57] Functions [1]: [partial_sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] @@ -534,17 +534,17 @@ Results [3]: [c_customer_sk#57, sum#61, isEmpty#62] Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] Arguments: hashpartitioning(c_customer_sk#57, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(94) CometColumnarToRow [codegen id : 4] +(94) CometNativeColumnarToRow Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -(95) HashAggregate [codegen id : 4] +(95) HashAggregate [codegen id : 2] Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] Keys [1]: [c_customer_sk#57] Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63] Results [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63 AS csales#64] -(96) HashAggregate [codegen id : 4] +(96) HashAggregate [codegen id : 2] Input [1]: [csales#64] Keys: [] Functions [1]: [partial_max(csales#64)] @@ -555,10 +555,10 @@ Results [1]: [max#66] Input [1]: [max#66] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(98) CometColumnarToRow [codegen id : 5] +(98) CometNativeColumnarToRow Input [1]: [max#66] -(99) HashAggregate [codegen id : 5] +(99) HashAggregate [codegen id : 3] Input [1]: [max#66] Keys: [] Functions [1]: [max(csales#64)] @@ -567,7 +567,7 @@ Results [1]: [max(csales#64)#67 AS tpcds_cmax#68] Subquery:4 Hosting operator id = 80 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#56 BroadcastExchange (104) -+- * CometColumnarToRow (103) ++- CometNativeColumnarToRow (103) +- CometProject (102) +- CometFilter (101) +- CometNativeScan parquet spark_catalog.default.date_dim (100) @@ -588,7 +588,7 @@ Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#58)) Input [2]: [d_date_sk#58, d_year#69] Arguments: [d_date_sk#58], [d_date_sk#58] -(103) CometColumnarToRow [codegen id : 1] +(103) CometNativeColumnarToRow Input [1]: [d_date_sk#58] (104) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt index aadeb13f7b..7040e78da1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -23,7 +23,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -35,17 +35,17 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort @@ -53,11 +53,11 @@ CometColumnarToRow : : +- CometFilter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -69,16 +69,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -112,7 +112,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -124,17 +124,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/simplified.txt index d42a6ba29e..1a34e6b5b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/simplified.txt @@ -1,142 +1,130 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - 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] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] +CometNativeColumnarToRow + CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] + CometExchange #1 + 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] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + WholeStageCodegen (3) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (1) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (3) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_customer_sk] #11 + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - 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] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #15 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [d_date_sk] #14 + BroadcastExchange #13 + CometNativeColumnarToRow + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + ReusedExchange [d_date_sk] #12 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometExchange [c_customer_sk] #8 + 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] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #9 + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_quantity,ws_list_price] [sales] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #15 + WholeStageCodegen (6) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/explain.txt index ce21e141e4..1daaeb8fae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/explain.txt @@ -377,7 +377,7 @@ Input [1]: [sum(sales)#50] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 Input [3]: [d_date_sk#29, d_year#30, d_moy#31] Arguments: [d_date_sk#29], [d_date_sk#29] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -541,7 +541,7 @@ Input [1]: [tpcds_cmax#63] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 BroadcastExchange (100) -+- * CometColumnarToRow (99) ++- CometNativeColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) Input [2]: [d_date_sk#57, d_year#58] Arguments: [d_date_sk#57], [d_date_sk#57] -(99) CometColumnarToRow [codegen id : 1] +(99) CometNativeColumnarToRow Input [1]: [d_date_sk#57] (100) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt index a84226b077..0d7391c0d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,7 +112,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/simplified.txt index 8f1bddf6c0..38f04e8e26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/simplified.txt @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -36,12 +34,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt index ce21e141e4..1daaeb8fae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt @@ -377,7 +377,7 @@ Input [1]: [sum(sales)#50] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 Input [3]: [d_date_sk#29, d_year#30, d_moy#31] Arguments: [d_date_sk#29], [d_date_sk#29] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -541,7 +541,7 @@ Input [1]: [tpcds_cmax#63] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 BroadcastExchange (100) -+- * CometColumnarToRow (99) ++- CometNativeColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) Input [2]: [d_date_sk#57, d_year#58] Arguments: [d_date_sk#57], [d_date_sk#57] -(99) CometColumnarToRow [codegen id : 1] +(99) CometNativeColumnarToRow Input [1]: [d_date_sk#57] (100) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/extended.txt index a84226b077..0d7391c0d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,7 +112,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 8f1bddf6c0..38f04e8e26 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 @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -36,12 +34,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/explain.txt index 6003575bd0..e8d8b21599 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (90) +CometNativeColumnarToRow (90) +- CometTakeOrderedAndProject (89) +- CometUnion (88) :- CometHashAggregate (64) @@ -21,7 +21,7 @@ : : : : +- * Project (21) : : : : +- * Filter (20) : : : : +- * HashAggregate (19) - : : : : +- * CometColumnarToRow (18) + : : : : +- CometNativeColumnarToRow (18) : : : : +- CometColumnarExchange (17) : : : : +- * HashAggregate (16) : : : : +- * Project (15) @@ -33,7 +33,7 @@ : : : : : : +- Scan parquet spark_catalog.default.store_sales (4) : : : : : +- ReusedExchange (7) : : : : +- BroadcastExchange (13) - : : : : +- * CometColumnarToRow (12) + : : : : +- CometNativeColumnarToRow (12) : : : : +- CometFilter (11) : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : +- CometSort (40) @@ -99,10 +99,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Condition : isnotnull(cs_bill_customer_sk#1) @@ -114,23 +114,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 3] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(6) Filter [codegen id : 3] +(6) Filter [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) (7) ReusedExchange [Reuses operator id: 100] Output [2]: [d_date_sk#10, d_date#11] -(8) BroadcastHashJoin [codegen id : 3] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(9) Project [codegen id : 3] +(9) Project [codegen id : 1] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] @@ -145,24 +145,24 @@ ReadSchema: struct Input [2]: [i_item_sk#12, i_item_desc#13] Condition : isnotnull(i_item_sk#12) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [i_item_sk#12, i_item_desc#13] (13) BroadcastExchange Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(14) BroadcastHashJoin [codegen id : 3] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 3] +(15) Project [codegen id : 1] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(16) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 1] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] @@ -173,21 +173,21 @@ Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(18) CometColumnarToRow [codegen id : 4] +(18) CometNativeColumnarToRow Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 2] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(20) Filter [codegen id : 4] +(20) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(21) Project [codegen id : 4] +(21) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] @@ -195,13 +195,13 @@ Input [2]: [item_sk#18, cnt#19] Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 5] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 5] +(24) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -394,23 +394,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 10] +(66) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -(67) Filter [codegen id : 10] +(67) Filter [codegen id : 6] Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] Condition : isnotnull(ws_bill_customer_sk#41) (68) ReusedExchange [Reuses operator id: 22] Output [1]: [item_sk#45] -(69) BroadcastHashJoin [codegen id : 10] +(69) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#40] Right keys [1]: [item_sk#45] Join type: LeftSemi Join condition: None -(70) Project [codegen id : 10] +(70) Project [codegen id : 6] Output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] @@ -493,14 +493,14 @@ Child 1 Input [3]: [c_last_name#54, c_first_name#53, sales#59] Input [3]: [c_last_name#34, c_first_name#33, sales#58] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#34 ASC NULLS FIRST,c_first_name#33 ASC NULLS FIRST,sales#58 ASC NULLS FIRST], output=[c_last_name#34,c_first_name#33,sales#58]), [c_last_name#34, c_first_name#33, sales#58], 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, sales#58] -(90) CometColumnarToRow [codegen id : 11] +(90) CometNativeColumnarToRow Input [3]: [c_last_name#34, c_first_name#33, sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (95) -+- * CometColumnarToRow (94) ++- CometNativeColumnarToRow (94) +- CometProject (93) +- CometFilter (92) +- CometNativeScan parquet spark_catalog.default.date_dim (91) @@ -521,7 +521,7 @@ Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2 Input [3]: [d_date_sk#35, d_year#36, d_moy#37] Arguments: [d_date_sk#35], [d_date_sk#35] -(94) CometColumnarToRow [codegen id : 1] +(94) CometNativeColumnarToRow Input [1]: [d_date_sk#35] (95) BroadcastExchange @@ -530,7 +530,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (100) -+- * CometColumnarToRow (99) ++- CometNativeColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometNativeScan parquet spark_catalog.default.date_dim (96) @@ -551,7 +551,7 @@ Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#60] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(99) CometColumnarToRow [codegen id : 1] +(99) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (100) BroadcastExchange @@ -560,11 +560,11 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#28, [id=#29] * HashAggregate (120) -+- * CometColumnarToRow (119) ++- CometNativeColumnarToRow (119) +- CometColumnarExchange (118) +- * HashAggregate (117) +- * HashAggregate (116) - +- * CometColumnarToRow (115) + +- CometNativeColumnarToRow (115) +- CometColumnarExchange (114) +- * HashAggregate (113) +- * Project (112) @@ -575,7 +575,7 @@ Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquer : : +- * ColumnarToRow (102) : : +- Scan parquet spark_catalog.default.store_sales (101) : +- BroadcastExchange (107) - : +- * CometColumnarToRow (106) + : +- CometNativeColumnarToRow (106) : +- CometFilter (105) : +- CometNativeScan parquet spark_catalog.default.customer (104) +- ReusedExchange (110) @@ -589,10 +589,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#64), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 3] +(102) ColumnarToRow [codegen id : 1] Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] -(103) Filter [codegen id : 3] +(103) Filter [codegen id : 1] Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] Condition : isnotnull(ss_customer_sk#61) @@ -607,37 +607,37 @@ ReadSchema: struct Input [1]: [c_customer_sk#66] Condition : isnotnull(c_customer_sk#66) -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [c_customer_sk#66] (107) BroadcastExchange Input [1]: [c_customer_sk#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(108) BroadcastHashJoin [codegen id : 3] +(108) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#61] Right keys [1]: [c_customer_sk#66] Join type: Inner Join condition: None -(109) Project [codegen id : 3] +(109) Project [codegen id : 1] Output [4]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] Input [5]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] (110) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#67] -(111) BroadcastHashJoin [codegen id : 3] +(111) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#64] Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None -(112) Project [codegen id : 3] +(112) Project [codegen id : 1] Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] Input [5]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66, d_date_sk#67] -(113) HashAggregate [codegen id : 3] +(113) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] Keys [1]: [c_customer_sk#66] Functions [1]: [partial_sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] @@ -648,17 +648,17 @@ Results [3]: [c_customer_sk#66, sum#70, isEmpty#71] Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] Arguments: hashpartitioning(c_customer_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(115) CometColumnarToRow [codegen id : 4] +(115) CometNativeColumnarToRow Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] -(116) HashAggregate [codegen id : 4] +(116) HashAggregate [codegen id : 2] Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] Keys [1]: [c_customer_sk#66] Functions [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] Aggregate Attributes [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72] Results [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72 AS csales#73] -(117) HashAggregate [codegen id : 4] +(117) HashAggregate [codegen id : 2] Input [1]: [csales#73] Keys: [] Functions [1]: [partial_max(csales#73)] @@ -669,10 +669,10 @@ Results [1]: [max#75] Input [1]: [max#75] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(119) CometColumnarToRow [codegen id : 5] +(119) CometNativeColumnarToRow Input [1]: [max#75] -(120) HashAggregate [codegen id : 5] +(120) HashAggregate [codegen id : 3] Input [1]: [max#75] Keys: [] Functions [1]: [max(csales#73)] @@ -681,7 +681,7 @@ Results [1]: [max(csales#73)#76 AS tpcds_cmax#77] Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65 BroadcastExchange (125) -+- * CometColumnarToRow (124) ++- CometNativeColumnarToRow (124) +- CometProject (123) +- CometFilter (122) +- CometNativeScan parquet spark_catalog.default.date_dim (121) @@ -702,7 +702,7 @@ Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#67)) Input [2]: [d_date_sk#67, d_year#78] Arguments: [d_date_sk#67], [d_date_sk#67] -(124) CometColumnarToRow [codegen id : 1] +(124) CometNativeColumnarToRow Input [1]: [d_date_sk#67] (125) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt index d41bf0802a..188775e7df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometUnion :- CometHashAggregate @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -26,7 +26,7 @@ CometColumnarToRow : : : : +- Project : : : : +- Filter : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project @@ -38,17 +38,17 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- CometSort @@ -56,11 +56,11 @@ CometColumnarToRow : : : +- CometFilter : : : : +- Subquery : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project @@ -72,16 +72,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -154,17 +154,17 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/simplified.txt index 41f01311f8..d543da014a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/simplified.txt @@ -1,164 +1,152 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - 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 [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] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + CometTakeOrderedAndProject [c_last_name,c_first_name,sales] + CometUnion [c_last_name,c_first_name,sales] + 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 [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] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + WholeStageCodegen (3) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (1) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (3) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_customer_sk] #11 + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - 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] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #14 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #15 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - CometBroadcastExchange [d_date_sk] #16 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] #17 - 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] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #18 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 + BroadcastExchange #13 + CometNativeColumnarToRow + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + ReusedExchange [d_date_sk] #12 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometExchange [c_customer_sk] #8 + 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] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #9 + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #14 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] + CometSort [c_customer_sk,c_first_name,c_last_name] + CometExchange [c_customer_sk] #15 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] CometSort [c_customer_sk] CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 - ReusedExchange [d_date_sk] #16 + CometBroadcastExchange [d_date_sk] #16 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + 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] #17 + 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] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #18 + WholeStageCodegen (6) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 + ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/explain.txt index 474a708a44..3d8d8a3361 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/explain.txt @@ -491,7 +491,7 @@ Input [3]: [c_last_name#33, c_first_name#32, sales#58] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (93) -+- * CometColumnarToRow (92) ++- CometNativeColumnarToRow (92) +- CometProject (91) +- CometFilter (90) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) @@ -512,7 +512,7 @@ Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2 Input [3]: [d_date_sk#34, d_year#35, d_moy#36] Arguments: [d_date_sk#34], [d_date_sk#34] -(92) CometColumnarToRow [codegen id : 1] +(92) CometNativeColumnarToRow Input [1]: [d_date_sk#34] (93) BroadcastExchange @@ -521,7 +521,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometProject (96) +- CometFilter (95) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) @@ -542,7 +542,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (98) BroadcastExchange @@ -655,7 +655,7 @@ Input [1]: [tpcds_cmax#72] Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -676,7 +676,7 @@ Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) Input [2]: [d_date_sk#66, d_year#67] Arguments: [d_date_sk#66], [d_date_sk#66] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#66] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt index cd91f5de49..4838389d3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,7 +35,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/simplified.txt index 92563114ae..b81ee0b822 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -39,12 +37,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -73,12 +69,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt index 474a708a44..3d8d8a3361 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt @@ -491,7 +491,7 @@ Input [3]: [c_last_name#33, c_first_name#32, sales#58] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (93) -+- * CometColumnarToRow (92) ++- CometNativeColumnarToRow (92) +- CometProject (91) +- CometFilter (90) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) @@ -512,7 +512,7 @@ Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2 Input [3]: [d_date_sk#34, d_year#35, d_moy#36] Arguments: [d_date_sk#34], [d_date_sk#34] -(92) CometColumnarToRow [codegen id : 1] +(92) CometNativeColumnarToRow Input [1]: [d_date_sk#34] (93) BroadcastExchange @@ -521,7 +521,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometProject (96) +- CometFilter (95) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) @@ -542,7 +542,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (98) BroadcastExchange @@ -655,7 +655,7 @@ Input [1]: [tpcds_cmax#72] Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -676,7 +676,7 @@ Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) Input [2]: [d_date_sk#66, d_year#67] Arguments: [d_date_sk#66], [d_date_sk#66] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#66] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/extended.txt index cd91f5de49..4838389d3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,7 +35,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 92563114ae..b81ee0b822 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -39,12 +37,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -73,12 +69,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/explain.txt index 67a43617c7..6d8ba14019 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == * Filter (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) + :- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometProject (24) @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometNativeColumnarToRow Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometNativeScan parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometNativeColumnarToRow Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -272,16 +272,16 @@ Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subqu Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometColumnarExchange (75) +- * HashAggregate (74) +- * HashAggregate (73) - +- * CometColumnarToRow (72) + +- CometNativeColumnarToRow (72) +- CometColumnarExchange (71) +- * HashAggregate (70) +- * Project (69) +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) + :- CometNativeColumnarToRow (66) : +- CometProject (65) : +- CometBroadcastHashJoin (64) : :- CometProject (62) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometNativeColumnarToRow Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#69] Keys: [] Functions [1]: [partial_avg(netpaid#69)] @@ -426,10 +426,10 @@ Results [2]: [sum#72, count#73] Input [2]: [sum#72, count#73] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometNativeColumnarToRow Input [2]: [sum#72, count#73] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#72, count#73] Keys: [] Functions [1]: [avg(netpaid#69)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/extended.txt index b65f56f327..03e6809902 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/extended.txt @@ -1,16 +1,16 @@ Filter : +- Subquery : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow +: :- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometProject @@ -42,21 +42,21 @@ Filter : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometProject @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/simplified.txt index af8d5ee7aa..affed709e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -39,21 +39,21 @@ WholeStageCodegen (4) InputAdapter ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/explain.txt index c6e3a30a1f..7574af84f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#69] Keys: [] Functions [1]: [partial_avg(netpaid#69)] @@ -426,10 +426,10 @@ Results [2]: [sum#72, count#73] Input [2]: [sum#72, count#73] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#72, count#73] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#72, count#73] Keys: [] Functions [1]: [avg(netpaid#69)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt index c6e3a30a1f..7574af84f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#69] Keys: [] Functions [1]: [partial_avg(netpaid#69)] @@ -426,10 +426,10 @@ Results [2]: [sum#72, count#73] Input [2]: [sum#72, count#73] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#72, count#73] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#72, count#73] Keys: [] Functions [1]: [avg(netpaid#69)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/explain.txt index 65c5d9d521..78f508dfa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == * Filter (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) + :- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometProject (24) @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometNativeColumnarToRow Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometNativeScan parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometNativeColumnarToRow Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -272,16 +272,16 @@ Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subqu Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometColumnarExchange (75) +- * HashAggregate (74) +- * HashAggregate (73) - +- * CometColumnarToRow (72) + +- CometNativeColumnarToRow (72) +- CometColumnarExchange (71) +- * HashAggregate (70) +- * Project (69) +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) + :- CometNativeColumnarToRow (66) : +- CometProject (65) : +- CometBroadcastHashJoin (64) : :- CometProject (62) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometNativeColumnarToRow Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#69] Keys: [] Functions [1]: [partial_avg(netpaid#69)] @@ -426,10 +426,10 @@ Results [2]: [sum#72, count#73] Input [2]: [sum#72, count#73] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometNativeColumnarToRow Input [2]: [sum#72, count#73] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#72, count#73] Keys: [] Functions [1]: [avg(netpaid#69)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/extended.txt index b65f56f327..03e6809902 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/extended.txt @@ -1,16 +1,16 @@ Filter : +- Subquery : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow +: :- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometProject @@ -42,21 +42,21 @@ Filter : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometProject @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/simplified.txt index af8d5ee7aa..affed709e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -39,21 +39,21 @@ WholeStageCodegen (4) InputAdapter ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/explain.txt index a77a3f88b3..76ff440e15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#69] Keys: [] Functions [1]: [partial_avg(netpaid#69)] @@ -426,10 +426,10 @@ Results [2]: [sum#72, count#73] Input [2]: [sum#72, count#73] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#72, count#73] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#72, count#73] Keys: [] Functions [1]: [avg(netpaid#69)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt index a77a3f88b3..76ff440e15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#69] Keys: [] Functions [1]: [partial_avg(netpaid#69)] @@ -426,10 +426,10 @@ Results [2]: [sum#72, count#73] Input [2]: [sum#72, count#73] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#72, count#73] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#72, count#73] Keys: [] Functions [1]: [avg(netpaid#69)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/explain.txt index 15bdd29f61..17e2a46a14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- ReusedExchange (19) : : +- ReusedExchange (22) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.store (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.item (32) @@ -52,10 +52,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) @@ -78,13 +78,13 @@ Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnot Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 8] +(8) BroadcastHashJoin [codegen id : 3] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] Join type: Inner Join condition: None -(9) Project [codegen id : 8] +(9) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] @@ -107,52 +107,52 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 3] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] Join type: Inner Join condition: None -(15) Project [codegen id : 8] +(15) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] (16) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#18] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(18) Project [codegen id : 3] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#18] (19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#19] -(20) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_returned_date_sk#12] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(21) Project [codegen id : 8] +(21) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] (22) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(24) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] @@ -171,20 +171,20 @@ Condition : isnotnull(s_store_sk#21) Input [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] Arguments: [s_store_sk#21, s_store_id#24, s_store_name#23], [s_store_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#22, 16, true, false, true) AS s_store_id#24, s_store_name#23] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] (29) BroadcastExchange Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#21, s_store_id#24, s_store_name#23] @@ -203,24 +203,24 @@ Condition : isnotnull(i_item_sk#25) Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] Arguments: [i_item_sk#25, i_item_id#28, i_item_desc#27], [i_item_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#26, 16, true, false, true) AS i_item_id#28, i_item_desc#27] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] (36) BroadcastExchange Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#25] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(38) Project [codegen id : 3] Output [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_sk#25, i_item_id#28, i_item_desc#27] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 3] Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] @@ -231,10 +231,10 @@ Results [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum# Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] Arguments: hashpartitioning(i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 4] Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] @@ -249,7 +249,7 @@ Arguments: 100, [i_item_id#28 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 4) Input [3]: [d_date_sk#18, d_year#41, d_moy#42] Arguments: [d_date_sk#18], [d_date_sk#18] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#18] (48) BroadcastExchange @@ -279,7 +279,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -300,7 +300,7 @@ Condition : (((((isnotnull(d_moy#44) AND isnotnull(d_year#43)) AND (d_moy#44 >= Input [3]: [d_date_sk#19, d_year#43, d_moy#44] Arguments: [d_date_sk#19], [d_date_sk#19] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt index 26e924fd7e..15c6bec5dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,27 +42,27 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/simplified.txt index 288561a740..cc47e03799 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,12 +39,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -63,17 +59,13 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk,i_item_id,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/explain.txt index 6a9f9094d3..06156ea810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/explain.txt @@ -256,7 +256,7 @@ Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/simplified.txt index dcad304452..7d8d4694e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/simplified.txt @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt index 6a9f9094d3..06156ea810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt @@ -256,7 +256,7 @@ Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 dcad304452..7d8d4694e8 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 @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/explain.txt index fb4c487ea9..069d2d52eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * HashAggregate (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * Project (27) @@ -16,18 +16,18 @@ TakeOrderedAndProject (32) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : +- ReusedExchange (11) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.item (14) +- BroadcastExchange (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometNativeScan parquet spark_catalog.default.promotion (21) @@ -41,10 +41,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) @@ -63,33 +63,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#1] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] @@ -108,20 +108,20 @@ Condition : isnotnull(i_item_sk#15) Input [2]: [i_item_sk#15, i_item_id#16] Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#16, 16, true, false, true) AS i_item_id#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [i_item_sk#15, i_item_id#17] (18) BroadcastExchange Input [2]: [i_item_sk#15, i_item_id#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#2] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#15, i_item_id#17] @@ -140,24 +140,24 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Arguments: [p_promo_sk#18], [p_promo_sk#18] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [1]: [p_promo_sk#18] (25) BroadcastExchange Input [1]: [p_promo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_promo_sk#3] Right keys [1]: [p_promo_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] @@ -168,10 +168,10 @@ Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34 Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Keys [1]: [i_item_id#17] Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] @@ -186,7 +186,7 @@ Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#45] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt index 3b9d40795e..ceba8f5828 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -16,27 +16,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/simplified.txt index 7eb9c67611..23b7e6a8f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] BroadcastHashJoin [cs_promo_sk,p_promo_sk] @@ -20,35 +20,27 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometNativeColumnarToRow + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/explain.txt index 3d70460689..ae35309ada 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt index dbea7afe34..ea785136aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/simplified.txt index 0b56a47547..f040250da3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/simplified.txt @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt index 3d70460689..ae35309ada 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/extended.txt index dbea7afe34..ea785136aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0b56a47547..f040250da3 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 @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/explain.txt index b9e3e82684..525997db60 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (33) +- * HashAggregate (32) - +- * CometColumnarToRow (31) + +- CometNativeColumnarToRow (31) +- CometColumnarExchange (30) +- * HashAggregate (29) +- * Expand (28) @@ -17,18 +17,18 @@ TakeOrderedAndProject (33) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : +- ReusedExchange (11) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.store (14) +- BroadcastExchange (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometNativeScan parquet spark_catalog.default.item (21) @@ -42,10 +42,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) @@ -64,33 +64,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 38] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -109,20 +109,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#15, s_state#16] Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [s_store_sk#15, s_state#17] (18) BroadcastExchange Input [2]: [s_store_sk#15, s_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] @@ -141,28 +141,28 @@ Condition : isnotnull(i_item_sk#18) Input [2]: [i_item_sk#18, i_item_id#19] Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [2]: [i_item_sk#18, i_item_id#20] (25) BroadcastExchange Input [2]: [i_item_sk#18, i_item_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] -(28) Expand [codegen id : 5] +(28) Expand [codegen id : 1] Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] -(29) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 1] Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] @@ -173,10 +173,10 @@ Results [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] Arguments: hashpartitioning(i_item_id#21, s_state#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(31) CometColumnarToRow [codegen id : 6] +(31) CometNativeColumnarToRow Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 2] Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] @@ -191,7 +191,7 @@ Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#49] Arguments: [d_date_sk#14], [d_date_sk#14] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt index a4946e4771..20f7517d3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -17,27 +17,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/simplified.txt index dcb4953707..452cc03fb3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] @@ -21,35 +21,27 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/explain.txt index eb158b2889..87c8eb2bc3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/explain.txt @@ -186,7 +186,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt index f550f3855c..bbfc91ed08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/simplified.txt index bdae0cc477..98686d7452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt index eb158b2889..87c8eb2bc3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt @@ -186,7 +186,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/extended.txt index f550f3855c..bbfc91ed08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 bdae0cc477..98686d7452 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/explain.txt index 985274408b..bb87d4cc2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/explain.txt @@ -4,72 +4,72 @@ : :- * BroadcastNestedLoopJoin Inner BuildRight (50) : : :- * BroadcastNestedLoopJoin Inner BuildRight (37) : : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) +: : : : :- CometNativeColumnarToRow (11) : : : : : +- CometHashAggregate (10) : : : : : +- CometColumnarExchange (9) : : : : : +- * HashAggregate (8) : : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) +: : : : : +- CometNativeColumnarToRow (6) : : : : : +- CometExchange (5) : : : : : +- CometHashAggregate (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) : : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) +: : : : +- CometNativeColumnarToRow (22) : : : : +- CometHashAggregate (21) : : : : +- CometColumnarExchange (20) : : : : +- * HashAggregate (19) : : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) +: : : : +- CometNativeColumnarToRow (17) : : : : +- CometExchange (16) : : : : +- CometHashAggregate (15) : : : : +- CometProject (14) : : : : +- CometFilter (13) : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (12) : : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) +: : : +- CometNativeColumnarToRow (35) : : : +- CometHashAggregate (34) : : : +- CometColumnarExchange (33) : : : +- * HashAggregate (32) : : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) +: : : +- CometNativeColumnarToRow (30) : : : +- CometExchange (29) : : : +- CometHashAggregate (28) : : : +- CometProject (27) : : : +- CometFilter (26) : : : +- CometNativeScan parquet spark_catalog.default.store_sales (25) : : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) +: : +- CometNativeColumnarToRow (48) : : +- CometHashAggregate (47) : : +- CometColumnarExchange (46) : : +- * HashAggregate (45) : : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) +: : +- CometNativeColumnarToRow (43) : : +- CometExchange (42) : : +- CometHashAggregate (41) : : +- CometProject (40) : : +- CometFilter (39) : : +- CometNativeScan parquet spark_catalog.default.store_sales (38) : +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) +: +- CometNativeColumnarToRow (61) : +- CometHashAggregate (60) : +- CometColumnarExchange (59) : +- * HashAggregate (58) : +- * HashAggregate (57) -: +- * CometColumnarToRow (56) +: +- CometNativeColumnarToRow (56) : +- CometExchange (55) : +- CometHashAggregate (54) : +- CometProject (53) : +- CometFilter (52) : +- CometNativeScan parquet spark_catalog.default.store_sales (51) +- BroadcastExchange (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometHashAggregate (73) +- CometColumnarExchange (72) +- * HashAggregate (71) +- * HashAggregate (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometExchange (68) +- CometHashAggregate (67) +- CometProject (66) @@ -101,7 +101,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_li Input [4]: [ss_list_price#3, sum#6, count#7, count#8] Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [4]: [ss_list_price#3, sum#6, count#7, count#8] (7) HashAggregate [codegen id : 1] @@ -127,7 +127,7 @@ Input [4]: [sum#6, count#7, count#8, count#12] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -(11) CometColumnarToRow [codegen id : 12] +(11) CometNativeColumnarToRow Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] (12) CometNativeScan parquet spark_catalog.default.store_sales @@ -154,7 +154,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_l Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(17) CometColumnarToRow [codegen id : 2] +(17) CometNativeColumnarToRow Input [4]: [ss_list_price#18, sum#21, count#22, count#23] (18) HashAggregate [codegen id : 2] @@ -180,14 +180,14 @@ Input [4]: [sum#21, count#22, count#23, count#27] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] -(22) CometColumnarToRow [codegen id : 3] +(22) CometNativeColumnarToRow Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] (23) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(24) BroadcastNestedLoopJoin [codegen id : 12] +(24) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -215,17 +215,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_l Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(30) CometColumnarToRow [codegen id : 4] +(30) CometNativeColumnarToRow Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -(31) HashAggregate [codegen id : 4] +(31) HashAggregate [codegen id : 3] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys [1]: [ss_list_price#33] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] Results [4]: [ss_list_price#33, sum#36, count#37, count#38] -(32) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 3] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] @@ -241,14 +241,14 @@ Input [4]: [sum#36, count#37, count#38, count#42] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] -(35) CometColumnarToRow [codegen id : 5] +(35) CometNativeColumnarToRow Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] (36) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(37) BroadcastNestedLoopJoin [codegen id : 12] +(37) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -276,17 +276,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_l Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(43) CometColumnarToRow [codegen id : 6] +(43) CometNativeColumnarToRow Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -(44) HashAggregate [codegen id : 6] +(44) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys [1]: [ss_list_price#48] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] Results [4]: [ss_list_price#48, sum#51, count#52, count#53] -(45) HashAggregate [codegen id : 6] +(45) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] @@ -302,14 +302,14 @@ Input [4]: [sum#51, count#52, count#53, count#57] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] -(48) CometColumnarToRow [codegen id : 7] +(48) CometNativeColumnarToRow Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] (49) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(50) BroadcastNestedLoopJoin [codegen id : 12] +(50) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -337,17 +337,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_l Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(56) CometColumnarToRow [codegen id : 8] +(56) CometNativeColumnarToRow Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -(57) HashAggregate [codegen id : 8] +(57) HashAggregate [codegen id : 5] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys [1]: [ss_list_price#63] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] Results [4]: [ss_list_price#63, sum#66, count#67, count#68] -(58) HashAggregate [codegen id : 8] +(58) HashAggregate [codegen id : 5] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] @@ -363,14 +363,14 @@ Input [4]: [sum#66, count#67, count#68, count#72] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] -(61) CometColumnarToRow [codegen id : 9] +(61) CometNativeColumnarToRow Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] (62) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(63) BroadcastNestedLoopJoin [codegen id : 12] +(63) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -398,17 +398,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_l Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(69) CometColumnarToRow [codegen id : 10] +(69) CometNativeColumnarToRow Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -(70) HashAggregate [codegen id : 10] +(70) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys [1]: [ss_list_price#78] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] Results [4]: [ss_list_price#78, sum#81, count#82, count#83] -(71) HashAggregate [codegen id : 10] +(71) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] @@ -424,14 +424,14 @@ Input [4]: [sum#81, count#82, count#83, count#87] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] -(74) CometColumnarToRow [codegen id : 11] +(74) CometNativeColumnarToRow Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] (75) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(76) BroadcastNestedLoopJoin [codegen id : 12] +(76) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/extended.txt index 37c297f68a..d05989eb4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/extended.txt @@ -3,72 +3,72 @@ BroadcastNestedLoopJoin : :- BroadcastNestedLoopJoin : : :- BroadcastNestedLoopJoin : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : :- CometColumnarToRow +: : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate : : : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : : +- CometColumnarToRow +: : : : : +- CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometHashAggregate : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- BroadcastExchange -: : : : +- CometColumnarToRow +: : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : +- CometColumnarToRow +: : : : +- CometNativeColumnarToRow : : : : +- CometExchange : : : : +- CometHashAggregate : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- BroadcastExchange -: : : +- CometColumnarToRow +: : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : +- CometColumnarToRow +: : : +- CometNativeColumnarToRow : : : +- CometExchange : : : +- CometHashAggregate : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- BroadcastExchange -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometExchange : : +- CometHashAggregate : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometExchange : +- CometHashAggregate : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometExchange +- CometHashAggregate +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/simplified.txt index a8540a4abb..96a591d9ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (12) +WholeStageCodegen (7) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] CometColumnarExchange #1 WholeStageCodegen (1) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometExchange [ss_list_price] #2 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] @@ -20,86 +20,76 @@ WholeStageCodegen (12) CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #4 + WholeStageCodegen (2) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #5 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #7 + WholeStageCodegen (3) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #8 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) + InputAdapter + BroadcastExchange #9 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #10 + WholeStageCodegen (4) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #11 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #12 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #13 + WholeStageCodegen (5) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #14 + CometHashAggregate [ss_list_price,sum,count,count] + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #15 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #16 + WholeStageCodegen (6) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #17 + CometHashAggregate [ss_list_price,sum,count,count] + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/explain.txt index 20f1508413..43df55e471 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- ReusedExchange (19) : : +- ReusedExchange (22) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.store (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.item (32) @@ -52,10 +52,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) @@ -78,13 +78,13 @@ Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnot Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 8] +(8) BroadcastHashJoin [codegen id : 3] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] Join type: Inner Join condition: None -(9) Project [codegen id : 8] +(9) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] @@ -107,52 +107,52 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 3] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] Join type: Inner Join condition: None -(15) Project [codegen id : 8] +(15) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] (16) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#19] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(18) Project [codegen id : 3] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] (19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#20] -(20) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_returned_date_sk#12] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(21) Project [codegen id : 8] +(21) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] (22) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#21] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(24) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] @@ -171,20 +171,20 @@ Condition : isnotnull(s_store_sk#22) Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] Arguments: [s_store_sk#22, s_store_id#25, s_store_name#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#23, 16, true, false, true) AS s_store_id#25, s_store_name#24] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] (29) BroadcastExchange Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#22] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_store_id#25, s_store_name#24] @@ -203,24 +203,24 @@ Condition : isnotnull(i_item_sk#26) Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] Arguments: [i_item_sk#26, i_item_id#29, i_item_desc#28], [i_item_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#27, 16, true, false, true) AS i_item_id#29, i_item_desc#28] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] (36) BroadcastExchange Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(38) Project [codegen id : 3] Output [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_sk#26, i_item_id#29, i_item_desc#28] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 3] Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] @@ -231,10 +231,10 @@ Results [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum# Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] Arguments: hashpartitioning(i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 4] Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] @@ -249,7 +249,7 @@ Arguments: 100, [i_item_id#29 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 9) Input [3]: [d_date_sk#19, d_year#42, d_moy#43] Arguments: [d_date_sk#19], [d_date_sk#19] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (48) BroadcastExchange @@ -279,7 +279,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -300,7 +300,7 @@ Condition : (((((isnotnull(d_moy#45) AND isnotnull(d_year#44)) AND (d_moy#45 >= Input [3]: [d_date_sk#20, d_year#44, d_moy#45] Arguments: [d_date_sk#20], [d_date_sk#20] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#20] (53) BroadcastExchange @@ -309,7 +309,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (58) -+- * CometColumnarToRow (57) ++- CometNativeColumnarToRow (57) +- CometProject (56) +- CometFilter (55) +- CometNativeScan parquet spark_catalog.default.date_dim (54) @@ -330,7 +330,7 @@ Condition : (d_year#46 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) Input [2]: [d_date_sk#21, d_year#46] Arguments: [d_date_sk#21], [d_date_sk#21] -(57) CometColumnarToRow [codegen id : 1] +(57) CometNativeColumnarToRow Input [1]: [d_date_sk#21] (58) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt index dacaab515f..bd9fa52aa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,32 +42,32 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/simplified.txt index 54bdbeeb5a..19ffcce191 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,12 +39,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -56,12 +52,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -70,17 +64,13 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk,i_item_id,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/explain.txt index 88c51b72ee..58860a2993 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/explain.txt @@ -275,7 +275,7 @@ Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -296,7 +296,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (52) BroadcastExchange @@ -305,7 +305,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -326,7 +326,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (57) BroadcastExchange @@ -335,7 +335,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometFilter (59) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) @@ -356,7 +356,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#25] (62) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt index fe03fef8a7..8c47903ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/simplified.txt index 15b992d3a4..e772ae6f33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/simplified.txt @@ -23,34 +23,28 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt index 88c51b72ee..58860a2993 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt @@ -275,7 +275,7 @@ Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -296,7 +296,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (52) BroadcastExchange @@ -305,7 +305,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -326,7 +326,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (57) BroadcastExchange @@ -335,7 +335,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometFilter (59) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) @@ -356,7 +356,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#25] (62) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/extended.txt index fe03fef8a7..8c47903ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 15b992d3a4..e772ae6f33 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 @@ -23,34 +23,28 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/explain.txt index 9a5ab78af8..775ad33d71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/simplified.txt index c19e6caf52..0a6cce1eb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - 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 [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] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] + 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 [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] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/explain.txt index b3dd4f1415..421ec2ba5b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (54) : : +- * BroadcastHashJoin Inner BuildRight (38) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) @@ -20,18 +20,18 @@ TakeOrderedAndProject (54) : : : : : +- Scan parquet spark_catalog.default.web_returns (1) : : : : +- ReusedExchange (4) : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) + : : : +- CometNativeColumnarToRow (10) : : : +- CometProject (9) : : : +- CometFilter (8) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : : +- BroadcastExchange (37) : : +- * Filter (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) + : : +- CometNativeColumnarToRow (30) : : +- CometColumnarExchange (29) : : +- * HashAggregate (28) : : +- * Project (27) @@ -44,12 +44,12 @@ TakeOrderedAndProject (54) : : : +- ReusedExchange (22) : : +- ReusedExchange (25) : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometProject (42) : +- CometFilter (41) : +- CometNativeScan parquet spark_catalog.default.customer (40) +- BroadcastExchange (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometNativeScan parquet spark_catalog.default.customer_address (47) @@ -63,23 +63,23 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [wr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] @@ -98,24 +98,24 @@ Condition : (isnotnull(ca_address_sk#7) AND isnotnull(staticinvoke(class org.apa Input [2]: [ca_address_sk#7, ca_state#8] Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#9] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [ca_address_sk#7, ca_state#9] (11) BroadcastExchange Input [2]: [ca_address_sk#7, ca_state#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [wr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#9] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] @@ -126,17 +126,17 @@ Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 11] +(16) CometNativeColumnarToRow Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -(17) HashAggregate [codegen id : 11] +(17) HashAggregate [codegen id : 5] Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] -(18) Filter [codegen id : 11] +(18) Filter [codegen id : 5] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) @@ -148,40 +148,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#19), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 2] Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -(21) Filter [codegen id : 6] +(21) Filter [codegen id : 2] Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] Condition : isnotnull(wr_returning_addr_sk#17) (22) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(24) Project [codegen id : 2] Output [3]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19, d_date_sk#20] (25) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#21, ca_state#9] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_returning_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 2] Output [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#9] Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, ca_address_sk#21, ca_state#9] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#9] Keys [2]: [wr_returning_customer_sk#16, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#18))] @@ -192,17 +192,17 @@ Results [3]: [wr_returning_customer_sk#16, ca_state#9, sum#23] Input [3]: [wr_returning_customer_sk#16, ca_state#9, sum#23] Arguments: hashpartitioning(wr_returning_customer_sk#16, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [3]: [wr_returning_customer_sk#16, ca_state#9, sum#23] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [3]: [wr_returning_customer_sk#16, ca_state#9, sum#23] Keys [2]: [wr_returning_customer_sk#16, ca_state#9] Functions [1]: [sum(UnscaledValue(wr_return_amt#18))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#18))#12] Results [2]: [ca_state#9 AS ctr_state#24, MakeDecimal(sum(UnscaledValue(wr_return_amt#18))#12,17,2) AS ctr_total_return#25] -(32) HashAggregate [codegen id : 7] +(32) HashAggregate [codegen id : 3] Input [2]: [ctr_state#24, ctr_total_return#25] Keys [1]: [ctr_state#24] Functions [1]: [partial_avg(ctr_total_return#25)] @@ -213,17 +213,17 @@ Results [3]: [ctr_state#24, sum#28, count#29] Input [3]: [ctr_state#24, sum#28, count#29] Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 8] +(34) CometNativeColumnarToRow Input [3]: [ctr_state#24, sum#28, count#29] -(35) HashAggregate [codegen id : 8] +(35) HashAggregate [codegen id : 4] Input [3]: [ctr_state#24, sum#28, count#29] Keys [1]: [ctr_state#24] Functions [1]: [avg(ctr_total_return#25)] Aggregate Attributes [1]: [avg(ctr_total_return#25)#30] Results [2]: [(avg(ctr_total_return#25)#30 * 1.2) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#24] -(36) Filter [codegen id : 8] +(36) Filter [codegen id : 4] Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) @@ -231,13 +231,13 @@ Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 11] +(38) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_state#14] Right keys [1]: [ctr_state#24] Join type: Inner Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) -(39) Project [codegen id : 11] +(39) Project [codegen id : 5] Output [2]: [ctr_customer_sk#13, ctr_total_return#15] Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_state#24] @@ -256,20 +256,20 @@ Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] Arguments: [c_customer_sk#32, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#46, c_current_addr_sk#34, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#35, 10, true, false, true) AS c_salutation#47, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#36, 20, true, false, true) AS c_first_name#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#37, 30, true, false, true) AS c_last_name#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#38, 1, true, false, true) AS c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#43, 13, true, false, true) AS c_login#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#44, 50, true, false, true) AS c_email_address#52, c_last_review_date#45] -(43) CometColumnarToRow [codegen id : 9] +(43) CometNativeColumnarToRow Input [14]: [c_customer_sk#32, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45] (44) BroadcastExchange Input [14]: [c_customer_sk#32, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_customer_sk#13] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(46) Project [codegen id : 5] Output [14]: [ctr_total_return#15, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45] Input [16]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#32, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45] @@ -288,20 +288,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [ca_address_sk#53, ca_state#54] Arguments: [ca_address_sk#53], [ca_address_sk#53] -(50) CometColumnarToRow [codegen id : 10] +(50) CometNativeColumnarToRow Input [1]: [ca_address_sk#53] (51) BroadcastExchange Input [1]: [ca_address_sk#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#34] Right keys [1]: [ca_address_sk#53] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(53) Project [codegen id : 5] Output [13]: [c_customer_id#46, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45, ctr_total_return#15] Input [15]: [ctr_total_return#15, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45, ca_address_sk#53] @@ -313,7 +313,7 @@ Arguments: 100, [c_customer_id#46 ASC NULLS FIRST, c_salutation#47 ASC NULLS FIR Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (59) -+- * CometColumnarToRow (58) ++- CometNativeColumnarToRow (58) +- CometProject (57) +- CometFilter (56) +- CometNativeScan parquet spark_catalog.default.date_dim (55) @@ -334,7 +334,7 @@ Condition : ((isnotnull(d_year#55) AND (d_year#55 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#6, d_year#55] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(58) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (59) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt index 8373409822..c2e96eb412 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -19,28 +19,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -52,22 +52,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/simplified.txt index 72b6fbd1ba..4eacdafde1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (5) Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] @@ -8,10 +8,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] Project [wr_returning_customer_sk,wr_return_amt,ca_state] BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] @@ -23,37 +23,33 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre Scan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (8) + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] Project [wr_returning_customer_sk,wr_return_amt,ca_state] BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] @@ -70,17 +66,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometNativeColumnarToRow + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/explain.txt index 4c7261693d..ab92ad6015 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/explain.txt @@ -285,7 +285,7 @@ Input [13]: [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt index 7ebd69d340..70bff8abb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/simplified.txt index 0227534fa5..6eebcf2802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt index 4c7261693d..ab92ad6015 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt @@ -285,7 +285,7 @@ Input [13]: [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/extended.txt index 7ebd69d340..70bff8abb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0227534fa5..6eebcf2802 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/explain.txt index 9ce8c738e8..24cf234506 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (97) +CometNativeColumnarToRow (97) +- CometSort (96) +- CometColumnarExchange (95) +- * Project (94) @@ -11,7 +11,7 @@ : : : +- * BroadcastHashJoin Inner BuildRight (46) : : : :- * BroadcastHashJoin Inner BuildRight (31) : : : : :- * HashAggregate (16) - : : : : : +- * CometColumnarToRow (15) + : : : : : +- CometNativeColumnarToRow (15) : : : : : +- CometColumnarExchange (14) : : : : : +- * HashAggregate (13) : : : : : +- * Project (12) @@ -23,12 +23,12 @@ : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : : +- ReusedExchange (4) : : : : : +- BroadcastExchange (10) - : : : : : +- * CometColumnarToRow (9) + : : : : : +- CometNativeColumnarToRow (9) : : : : : +- CometFilter (8) : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : : : : +- BroadcastExchange (30) : : : : +- * HashAggregate (29) - : : : : +- * CometColumnarToRow (28) + : : : : +- CometNativeColumnarToRow (28) : : : : +- CometColumnarExchange (27) : : : : +- * HashAggregate (26) : : : : +- * Project (25) @@ -42,7 +42,7 @@ : : : : +- ReusedExchange (23) : : : +- BroadcastExchange (45) : : : +- * HashAggregate (44) - : : : +- * CometColumnarToRow (43) + : : : +- CometNativeColumnarToRow (43) : : : +- CometColumnarExchange (42) : : : +- * HashAggregate (41) : : : +- * Project (40) @@ -56,7 +56,7 @@ : : : +- ReusedExchange (38) : : +- BroadcastExchange (61) : : +- * HashAggregate (60) - : : +- * CometColumnarToRow (59) + : : +- CometNativeColumnarToRow (59) : : +- CometColumnarExchange (58) : : +- * HashAggregate (57) : : +- * Project (56) @@ -70,7 +70,7 @@ : : +- ReusedExchange (54) : +- BroadcastExchange (76) : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) + : +- CometNativeColumnarToRow (74) : +- CometColumnarExchange (73) : +- * HashAggregate (72) : +- * Project (71) @@ -84,7 +84,7 @@ : +- ReusedExchange (69) +- BroadcastExchange (92) +- * HashAggregate (91) - +- * CometColumnarToRow (90) + +- CometNativeColumnarToRow (90) +- CometColumnarExchange (89) +- * HashAggregate (88) +- * Project (87) @@ -106,23 +106,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_addr_sk#1) (4) ReusedExchange [Reuses operator id: 101] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] @@ -137,24 +137,24 @@ ReadSchema: struct Input [2]: [ca_address_sk#8, ca_county#9] Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) -(9) CometColumnarToRow [codegen id : 2] +(9) CometNativeColumnarToRow Input [2]: [ca_address_sk#8, ca_county#9] (10) BroadcastExchange Input [2]: [ca_address_sk#8, ca_county#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#1] Right keys [1]: [ca_address_sk#8] Join type: Inner Join condition: None -(12) Project [codegen id : 3] +(12) Project [codegen id : 1] Output [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 1] Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -165,10 +165,10 @@ Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(15) CometColumnarToRow [codegen id : 24] +(15) CometNativeColumnarToRow Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -(16) HashAggregate [codegen id : 24] +(16) HashAggregate [codegen id : 12] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -183,40 +183,40 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Condition : isnotnull(ss_addr_sk#14) (20) ReusedExchange [Reuses operator id: 105] Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#16] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] (23) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#21, ca_county#22] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_addr_sk#14] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] @@ -227,10 +227,10 @@ Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -(29) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 3] Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] @@ -241,7 +241,7 @@ Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15) Input [2]: [ca_county#22, store_sales#25] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 24] +(31) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#22] Join type: Inner @@ -255,40 +255,40 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] +(33) ColumnarToRow [codegen id : 4] Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -(34) Filter [codegen id : 10] +(34) Filter [codegen id : 4] Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Condition : isnotnull(ss_addr_sk#26) (35) ReusedExchange [Reuses operator id: 109] Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#28] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 4] Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] (38) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#33, ca_county#34] -(39) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_addr_sk#26] Right keys [1]: [ca_address_sk#33] Join type: Inner Join condition: None -(40) Project [codegen id : 10] +(40) Project [codegen id : 4] Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] -(41) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 4] Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] @@ -299,10 +299,10 @@ Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometColumnarToRow [codegen id : 11] +(43) CometNativeColumnarToRow Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 5] Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] @@ -313,13 +313,13 @@ Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27) Input [2]: [ca_county#34, store_sales#37] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 24] +(46) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#22] Right keys [1]: [ca_county#34] Join type: Inner Join condition: None -(47) Project [codegen id : 24] +(47) Project [codegen id : 12] Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] @@ -331,40 +331,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 14] +(49) ColumnarToRow [codegen id : 6] Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(50) Filter [codegen id : 14] +(50) Filter [codegen id : 6] Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Condition : isnotnull(ws_bill_addr_sk#38) (51) ReusedExchange [Reuses operator id: 101] Output [3]: [d_date_sk#41, d_year#42, d_qoy#43] -(52) BroadcastHashJoin [codegen id : 14] +(52) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#40] Right keys [1]: [d_date_sk#41] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 6] Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43] Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41, d_year#42, d_qoy#43] (54) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#44, ca_county#45] -(55) BroadcastHashJoin [codegen id : 14] +(55) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_bill_addr_sk#38] Right keys [1]: [ca_address_sk#44] Join type: Inner Join condition: None -(56) Project [codegen id : 14] +(56) Project [codegen id : 6] Output [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_address_sk#44, ca_county#45] -(57) HashAggregate [codegen id : 14] +(57) HashAggregate [codegen id : 6] Input [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] Keys [3]: [ca_county#45, d_qoy#43, d_year#42] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] @@ -375,10 +375,10 @@ Results [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] Arguments: hashpartitioning(ca_county#45, d_qoy#43, d_year#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(59) CometColumnarToRow [codegen id : 15] +(59) CometNativeColumnarToRow Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -(60) HashAggregate [codegen id : 15] +(60) HashAggregate [codegen id : 7] Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] Keys [3]: [ca_county#45, d_qoy#43, d_year#42] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] @@ -389,7 +389,7 @@ Results [2]: [ca_county#45, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39) Input [2]: [ca_county#45, web_sales#49] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(62) BroadcastHashJoin [codegen id : 24] +(62) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#45] Join type: Inner @@ -403,40 +403,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 18] +(64) ColumnarToRow [codegen id : 8] Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -(65) Filter [codegen id : 18] +(65) Filter [codegen id : 8] Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] Condition : isnotnull(ws_bill_addr_sk#50) (66) ReusedExchange [Reuses operator id: 105] Output [3]: [d_date_sk#53, d_year#54, d_qoy#55] -(67) BroadcastHashJoin [codegen id : 18] +(67) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#52] Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None -(68) Project [codegen id : 18] +(68) Project [codegen id : 8] Output [4]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55] Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54, d_qoy#55] (69) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#56, ca_county#57] -(70) BroadcastHashJoin [codegen id : 18] +(70) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_bill_addr_sk#50] Right keys [1]: [ca_address_sk#56] Join type: Inner Join condition: None -(71) Project [codegen id : 18] +(71) Project [codegen id : 8] Output [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_address_sk#56, ca_county#57] -(72) HashAggregate [codegen id : 18] +(72) HashAggregate [codegen id : 8] Input [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] Keys [3]: [ca_county#57, d_qoy#55, d_year#54] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#51))] @@ -447,10 +447,10 @@ Results [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] Arguments: hashpartitioning(ca_county#57, d_qoy#55, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(74) CometColumnarToRow [codegen id : 19] +(74) CometNativeColumnarToRow Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -(75) HashAggregate [codegen id : 19] +(75) HashAggregate [codegen id : 9] Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] Keys [3]: [ca_county#57, d_qoy#55, d_year#54] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#51))] @@ -461,13 +461,13 @@ Results [2]: [ca_county#57, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#51) Input [2]: [ca_county#57, web_sales#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] -(77) BroadcastHashJoin [codegen id : 24] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#45] Right keys [1]: [ca_county#57] Join type: Inner Join condition: (CASE WHEN (web_sales#49 > 0.00) THEN (web_sales#60 / web_sales#49) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) -(78) Project [codegen id : 24] +(78) Project [codegen id : 12] Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60] Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, ca_county#57, web_sales#60] @@ -479,40 +479,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 22] +(80) ColumnarToRow [codegen id : 10] Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -(81) Filter [codegen id : 22] +(81) Filter [codegen id : 10] Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] Condition : isnotnull(ws_bill_addr_sk#61) (82) ReusedExchange [Reuses operator id: 109] Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] -(83) BroadcastHashJoin [codegen id : 22] +(83) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#63] Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(84) Project [codegen id : 22] +(84) Project [codegen id : 10] Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65, d_qoy#66] (85) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#67, ca_county#68] -(86) BroadcastHashJoin [codegen id : 22] +(86) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_bill_addr_sk#61] Right keys [1]: [ca_address_sk#67] Join type: Inner Join condition: None -(87) Project [codegen id : 22] +(87) Project [codegen id : 10] Output [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_address_sk#67, ca_county#68] -(88) HashAggregate [codegen id : 22] +(88) HashAggregate [codegen id : 10] Input [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] Keys [3]: [ca_county#68, d_qoy#66, d_year#65] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] @@ -523,10 +523,10 @@ Results [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] Arguments: hashpartitioning(ca_county#68, d_qoy#66, d_year#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(90) CometColumnarToRow [codegen id : 23] +(90) CometNativeColumnarToRow Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -(91) HashAggregate [codegen id : 23] +(91) HashAggregate [codegen id : 11] Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] Keys [3]: [ca_county#68, d_qoy#66, d_year#65] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] @@ -537,13 +537,13 @@ Results [2]: [ca_county#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62) Input [2]: [ca_county#68, web_sales#71] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] -(93) BroadcastHashJoin [codegen id : 24] +(93) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#45] Right keys [1]: [ca_county#68] Join type: Inner Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#71 / web_sales#60) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) -(94) Project [codegen id : 24] +(94) Project [codegen id : 12] Output [6]: [ca_county#9, d_year#6, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#75] Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60, ca_county#68, web_sales#71] @@ -555,14 +555,14 @@ Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENT Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75], [ca_county#9 ASC NULLS FIRST] -(97) CometColumnarToRow [codegen id : 25] +(97) CometNativeColumnarToRow Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (101) -+- * CometColumnarToRow (100) ++- CometNativeColumnarToRow (100) +- CometFilter (99) +- CometNativeScan parquet spark_catalog.default.date_dim (98) @@ -578,7 +578,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(100) CometColumnarToRow [codegen id : 1] +(100) CometNativeColumnarToRow Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (101) BroadcastExchange @@ -587,7 +587,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 BroadcastExchange (105) -+- * CometColumnarToRow (104) ++- CometNativeColumnarToRow (104) +- CometFilter (103) +- CometNativeScan parquet spark_catalog.default.date_dim (102) @@ -603,7 +603,7 @@ ReadSchema: struct Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) -(104) CometColumnarToRow [codegen id : 1] +(104) CometNativeColumnarToRow Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] (105) BroadcastExchange @@ -612,7 +612,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (109) -+- * CometColumnarToRow (108) ++- CometNativeColumnarToRow (108) +- CometFilter (107) +- CometNativeScan parquet spark_catalog.default.date_dim (106) @@ -628,7 +628,7 @@ ReadSchema: struct Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) -(108) CometColumnarToRow [codegen id : 1] +(108) CometNativeColumnarToRow Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] (109) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt index f6f3c8032a..03bcf0413e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project @@ -10,7 +10,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- BroadcastHashJoin : : : : :- HashAggregate - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- HashAggregate : : : : : +- Project @@ -22,20 +22,20 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : : +- BroadcastExchange : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project @@ -47,20 +47,20 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- BroadcastExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -72,20 +72,20 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -97,16 +97,16 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -118,16 +118,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -139,11 +139,11 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/simplified.txt index f3a0572081..0a36656045 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/simplified.txt @@ -1,22 +1,75 @@ -WholeStageCodegen (25) - 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 - WholeStageCodegen (24) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] +CometNativeColumnarToRow + 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 + WholeStageCodegen (12) + Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] + Project [ca_county,d_year,store_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow + BroadcastHashJoin [ca_county,ca_county] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #2 + WholeStageCodegen (1) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [ca_address_sk,ca_county] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #6 + WholeStageCodegen (2) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #7 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #9 + WholeStageCodegen (4) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,d_year,d_qoy,ca_county] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -26,130 +79,45 @@ WholeStageCodegen (25) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #10 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #10 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (7) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #12 + WholeStageCodegen (6) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) + BroadcastExchange #13 + WholeStageCodegen (9) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #14 + WholeStageCodegen (8) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,d_year,d_qoy,ca_county] BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] @@ -159,8 +127,30 @@ WholeStageCodegen (25) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 + ReusedExchange [d_date_sk,d_year,d_qoy] #7 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (11) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #16 + WholeStageCodegen (10) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #10 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/explain.txt index 83d717e295..d300161c8f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/explain.txt @@ -511,7 +511,7 @@ Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#6 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (94) -+- * CometColumnarToRow (93) ++- CometNativeColumnarToRow (93) +- CometFilter (92) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) CometColumnarToRow [codegen id : 1] +(93) CometNativeColumnarToRow Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometFilter (96) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * CometColumnarToRow (101) ++- CometNativeColumnarToRow (101) +- CometFilter (100) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(101) CometColumnarToRow [codegen id : 1] +(101) CometNativeColumnarToRow Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt index b4e7aa4db6..780504fb09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/simplified.txt index 2f97384320..745f9356ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/simplified.txt @@ -22,11 +22,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -45,11 +43,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -66,11 +62,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt index 83d717e295..d300161c8f 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 @@ -511,7 +511,7 @@ Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#6 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (94) -+- * CometColumnarToRow (93) ++- CometNativeColumnarToRow (93) +- CometFilter (92) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) CometColumnarToRow [codegen id : 1] +(93) CometNativeColumnarToRow Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometFilter (96) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * CometColumnarToRow (101) ++- CometNativeColumnarToRow (101) +- CometFilter (100) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(101) CometColumnarToRow [codegen id : 1] +(101) CometNativeColumnarToRow Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/extended.txt index b4e7aa4db6..780504fb09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange 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 2f97384320..745f9356ce 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 @@ -22,11 +22,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -45,11 +43,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -66,11 +62,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/explain.txt index df3987b0a0..5405030155 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * HashAggregate (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * Project (27) @@ -13,14 +13,14 @@ : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : +- BroadcastExchange (8) - : : +- * CometColumnarToRow (7) + : : +- CometNativeColumnarToRow (7) : : +- CometProject (6) : : +- CometFilter (5) : : +- CometNativeScan parquet spark_catalog.default.item (4) : +- BroadcastExchange (22) : +- * Filter (21) : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) + : +- CometNativeColumnarToRow (19) : +- CometColumnarExchange (18) : +- * HashAggregate (17) : +- * Project (16) @@ -40,10 +40,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 3] Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 3] Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) @@ -62,20 +62,20 @@ Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotn Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [i_item_sk#5] (8) BroadcastExchange Input [1]: [i_item_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 6] +(9) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 6] +(10) Project [codegen id : 3] Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] @@ -87,27 +87,27 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] +(12) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -(13) Filter [codegen id : 3] +(13) Filter [codegen id : 1] Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] Condition : isnotnull(cs_item_sk#7) (14) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#10] -(15) BroadcastHashJoin [codegen id : 3] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(16) Project [codegen id : 3] +(16) Project [codegen id : 1] Output [2]: [cs_item_sk#7, cs_ext_discount_amt#8] Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#10] -(17) HashAggregate [codegen id : 3] +(17) HashAggregate [codegen id : 1] Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] Keys [1]: [cs_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] @@ -118,17 +118,17 @@ Results [3]: [cs_item_sk#7, sum#13, count#14] Input [3]: [cs_item_sk#7, sum#13, count#14] Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) CometColumnarToRow [codegen id : 4] +(19) CometNativeColumnarToRow Input [3]: [cs_item_sk#7, sum#13, count#14] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [cs_item_sk#7, sum#13, count#14] Keys [1]: [cs_item_sk#7] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#15] Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -(21) Filter [codegen id : 4] +(21) Filter [codegen id : 2] Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) @@ -136,30 +136,30 @@ Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#5] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16) -(24) Project [codegen id : 6] +(24) Project [codegen id : 3] Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] (25) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#17] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 3] Output [1]: [cs_ext_discount_amt#2] Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#17] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 3] Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] @@ -170,10 +170,10 @@ Results [1]: [sum#19] Input [1]: [sum#19] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [1]: [sum#19] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 4] Input [1]: [sum#19] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] @@ -184,7 +184,7 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#20,17,2) AS Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -205,7 +205,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#2 Input [2]: [d_date_sk#17, d_date#22] Arguments: [d_date_sk#17], [d_date_sk#17] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt index 1eb8f397c3..f7096f5c3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -13,19 +13,19 @@ HashAggregate : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -35,12 +35,12 @@ HashAggregate : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/simplified.txt index 14c21b32a5..c9760ec371 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [cs_ext_discount_amt] [sum,sum] Project [cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -17,29 +17,25 @@ WholeStageCodegen (7) Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(cs_ext_discount_amt))] HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cs_item_sk] #5 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] Project [cs_item_sk,cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/explain.txt index d6c10008ec..c4b57e9499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/explain.txt @@ -175,7 +175,7 @@ Input [1]: [excess discount amount#18] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt index 943bb147ff..c7273327fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/simplified.txt index 07619fc999..78be4baea7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/simplified.txt @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt index d6c10008ec..c4b57e9499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt @@ -175,7 +175,7 @@ Input [1]: [excess discount amount#18] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/extended.txt index 943bb147ff..c7273327fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 07619fc999..78be4baea7 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 @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/explain.txt index 0cc777296b..8e9915a8e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (66) +- * HashAggregate (65) - +- * CometColumnarToRow (64) + +- CometNativeColumnarToRow (64) +- CometColumnarExchange (63) +- * HashAggregate (62) +- Union (61) :- * HashAggregate (28) - : +- * CometColumnarToRow (27) + : +- CometNativeColumnarToRow (27) : +- CometColumnarExchange (26) : +- * HashAggregate (25) : +- * Project (24) @@ -20,12 +20,12 @@ TakeOrderedAndProject (66) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometBroadcastHashJoin (20) : :- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.item (14) @@ -34,7 +34,7 @@ TakeOrderedAndProject (66) : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.item (16) :- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -50,7 +50,7 @@ TakeOrderedAndProject (66) : : +- ReusedExchange (35) : +- ReusedExchange (38) +- * HashAggregate (60) - +- * CometColumnarToRow (59) + +- CometNativeColumnarToRow (59) +- CometColumnarExchange (58) +- * HashAggregate (57) +- * Project (56) @@ -75,23 +75,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] @@ -110,20 +110,20 @@ Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isno Input [2]: [ca_address_sk#7, ca_gmt_offset#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [ca_address_sk#7] (11) BroadcastExchange Input [1]: [ca_address_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] @@ -161,24 +161,24 @@ Left output [2]: [i_item_sk#9, i_manufact_id#10] Right output [1]: [i_manufact_id#12] Arguments: [i_manufact_id#10], [i_manufact_id#12], LeftSemi, BuildRight -(21) CometColumnarToRow [codegen id : 3] +(21) CometNativeColumnarToRow Input [2]: [i_item_sk#9, i_manufact_id#10] (22) BroadcastExchange Input [2]: [i_item_sk#9, i_manufact_id#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 4] +(23) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#9] Join type: Inner Join condition: None -(24) Project [codegen id : 4] +(24) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#3, i_manufact_id#10] Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_manufact_id#10] -(25) HashAggregate [codegen id : 4] +(25) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_manufact_id#10] Keys [1]: [i_manufact_id#10] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -189,10 +189,10 @@ Results [2]: [i_manufact_id#10, sum#14] Input [2]: [i_manufact_id#10, sum#14] Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(27) CometColumnarToRow [codegen id : 5] +(27) CometNativeColumnarToRow Input [2]: [i_manufact_id#10, sum#14] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 2] Input [2]: [i_manufact_id#10, sum#14] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -207,53 +207,53 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 3] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -(31) Filter [codegen id : 9] +(31) Filter [codegen id : 3] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) (32) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#21] -(33) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#20] Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None -(34) Project [codegen id : 9] +(34) Project [codegen id : 3] Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#21] (35) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#22] -(36) BroadcastHashJoin [codegen id : 9] +(36) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_addr_sk#17] Right keys [1]: [ca_address_sk#22] Join type: Inner Join condition: None -(37) Project [codegen id : 9] +(37) Project [codegen id : 3] Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#22] (38) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#23, i_manufact_id#24] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#18] Right keys [1]: [i_item_sk#23] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 3] Output [2]: [cs_ext_sales_price#19, i_manufact_id#24] Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#23, i_manufact_id#24] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 3] Input [2]: [cs_ext_sales_price#19, i_manufact_id#24] Keys [1]: [i_manufact_id#24] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] @@ -264,10 +264,10 @@ Results [2]: [i_manufact_id#24, sum#26] Input [2]: [i_manufact_id#24, sum#26] Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [2]: [i_manufact_id#24, sum#26] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 4] Input [2]: [i_manufact_id#24, sum#26] Keys [1]: [i_manufact_id#24] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] @@ -282,53 +282,53 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 14] +(46) ColumnarToRow [codegen id : 5] Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] -(47) Filter [codegen id : 14] +(47) Filter [codegen id : 5] Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) (48) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#33] -(49) BroadcastHashJoin [codegen id : 14] +(49) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#32] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(50) Project [codegen id : 14] +(50) Project [codegen id : 5] Output [3]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31] Input [5]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32, d_date_sk#33] (51) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#34] -(52) BroadcastHashJoin [codegen id : 14] +(52) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_bill_addr_sk#30] Right keys [1]: [ca_address_sk#34] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 5] Output [2]: [ws_item_sk#29, ws_ext_sales_price#31] Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ca_address_sk#34] (54) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#35, i_manufact_id#36] -(55) BroadcastHashJoin [codegen id : 14] +(55) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#35] Join type: Inner Join condition: None -(56) Project [codegen id : 14] +(56) Project [codegen id : 5] Output [2]: [ws_ext_sales_price#31, i_manufact_id#36] Input [4]: [ws_item_sk#29, ws_ext_sales_price#31, i_item_sk#35, i_manufact_id#36] -(57) HashAggregate [codegen id : 14] +(57) HashAggregate [codegen id : 5] Input [2]: [ws_ext_sales_price#31, i_manufact_id#36] Keys [1]: [i_manufact_id#36] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#31))] @@ -339,10 +339,10 @@ Results [2]: [i_manufact_id#36, sum#38] Input [2]: [i_manufact_id#36, sum#38] Arguments: hashpartitioning(i_manufact_id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(59) CometColumnarToRow [codegen id : 15] +(59) CometNativeColumnarToRow Input [2]: [i_manufact_id#36, sum#38] -(60) HashAggregate [codegen id : 15] +(60) HashAggregate [codegen id : 6] Input [2]: [i_manufact_id#36, sum#38] Keys [1]: [i_manufact_id#36] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#31))] @@ -351,7 +351,7 @@ Results [2]: [i_manufact_id#36, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price (61) Union -(62) HashAggregate [codegen id : 16] +(62) HashAggregate [codegen id : 7] Input [2]: [i_manufact_id#10, total_sales#16] Keys [1]: [i_manufact_id#10] Functions [1]: [partial_sum(total_sales#16)] @@ -362,10 +362,10 @@ Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(64) CometColumnarToRow [codegen id : 17] +(64) CometNativeColumnarToRow Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -(65) HashAggregate [codegen id : 17] +(65) HashAggregate [codegen id : 8] Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(total_sales#16)] @@ -380,7 +380,7 @@ Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_manufact_id#10, total_sales Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometNativeScan parquet spark_catalog.default.date_dim (67) @@ -401,7 +401,7 @@ Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1 Input [3]: [d_date_sk#6, d_year#47, d_moy#48] Arguments: [d_date_sk#6], [d_date_sk#6] -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt index 79b2700a8c..38b3efb123 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,22 +19,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -43,7 +43,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -57,17 +57,17 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -76,7 +76,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -90,17 +90,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/simplified.txt index fd1c1d0ef2..edbc778e0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_manufact_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -27,40 +27,34 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #6 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (10) + CometNativeColumnarToRow + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [i_manufact_id] #6 + CometProject [i_manufact_id] + CometFilter [i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #7 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -79,12 +73,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ReusedExchange [ca_address_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (15) + WholeStageCodegen (6) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] BroadcastHashJoin [ws_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/explain.txt index a178860ccc..ba765b93c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/explain.txt @@ -345,7 +345,7 @@ Input [2]: [i_manufact_id#12, total_sales#41] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * CometColumnarToRow (65) ++- CometNativeColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) @@ -366,7 +366,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) CometColumnarToRow [codegen id : 1] +(65) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt index 063fdb6faf..971bf3c1f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/simplified.txt index b4b84c2ac6..fc867ade9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt index a178860ccc..ba765b93c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt @@ -345,7 +345,7 @@ Input [2]: [i_manufact_id#12, total_sales#41] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * CometColumnarToRow (65) ++- CometNativeColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) @@ -366,7 +366,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) CometColumnarToRow [codegen id : 1] +(65) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/extended.txt index 063fdb6faf..971bf3c1f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 b4b84c2ac6..fc867ade9d 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/explain.txt index 18c904ceed..7f5f099e1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometSort (34) +- CometColumnarExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) :- * Filter (25) : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -20,17 +20,17 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.customer (26) @@ -44,23 +44,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] @@ -79,20 +79,20 @@ Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND is Input [2]: [s_store_sk#8, s_county#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (11) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] @@ -111,24 +111,24 @@ Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apac Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#10] (18) BroadcastExchange Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] @@ -139,17 +139,17 @@ Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) @@ -168,20 +168,20 @@ Condition : isnotnull(c_customer_sk#18) Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] (30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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] @@ -193,14 +193,14 @@ Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC 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: [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 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (40) -+- * CometColumnarToRow (39) ++- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometNativeScan parquet spark_catalog.default.date_dim (36) @@ -221,7 +221,7 @@ Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(39) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (40) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt index 9e4596c12c..40230582bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- BroadcastHashJoin :- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,27 +19,27 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/simplified.txt index 10af5ceadc..bbcdf458c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/simplified.txt @@ -1,59 +1,49 @@ -WholeStageCodegen (7) - 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 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] +CometNativeColumnarToRow + 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 + WholeStageCodegen (2) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/explain.txt index 287e323952..b1fe282f8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/simplified.txt index 47d0e35dd2..3362a73ddd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 287e323952..b1fe282f8b 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 @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 47d0e35dd2..3362a73ddd 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/explain.txt index 35d06d16ae..ef6172be1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -13,7 +13,7 @@ TakeOrderedAndProject (45) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -35,12 +35,12 @@ TakeOrderedAndProject (45) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_address (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) @@ -57,7 +57,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] (6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] @@ -87,7 +87,7 @@ Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi @@ -100,19 +100,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] (13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] @@ -120,7 +120,7 @@ Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) @@ -133,19 +133,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] @@ -153,17 +153,17 @@ Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(25) Filter [codegen id : 4] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -182,20 +182,20 @@ Condition : isnotnull(ca_address_sk#16) Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) AS ca_state#18] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [2]: [ca_address_sk#16, ca_state#18] (31) BroadcastExchange Input [2]: [ca_address_sk#16, ca_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, ca_state#18] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#18] @@ -214,24 +214,24 @@ Condition : isnotnull(cd_demo_sk#19) Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#20, 1, true, false, true) AS cd_gender#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] (38) BroadcastExchange Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 4] Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Input [8]: [c_current_cdemo_sk#4, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 4] Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [partial_count(1), partial_min(cd_dep_count#22), partial_max(cd_dep_count#22), partial_avg(cd_dep_count#22), partial_min(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_avg(cd_dep_employed_count#23), partial_min(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_avg(cd_dep_college_count#24)] @@ -242,10 +242,10 @@ Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 5] Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [count(1), min(cd_dep_count#22), max(cd_dep_count#22), avg(cd_dep_count#22), min(cd_dep_employed_count#23), max(cd_dep_employed_count#23), avg(cd_dep_employed_count#23), min(cd_dep_college_count#24), max(cd_dep_college_count#24), avg(cd_dep_college_count#24)] @@ -260,7 +260,7 @@ Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * CometColumnarToRow (49) ++- CometNativeColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometNativeScan parquet spark_catalog.default.date_dim (46) @@ -281,7 +281,7 @@ Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 2 Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt index 8935b6cc13..40a8ef4a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -22,12 +22,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,17 +49,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/simplified.txt index 48bc5637d0..a0b0348e6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -29,17 +29,15 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,7 +48,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -61,17 +59,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/explain.txt index fedc50de1e..61463feb76 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#20) Input [2]: [ca_address_sk#20, ca_state#21] Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#20, ca_state#22] (33) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#22] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#23) Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/simplified.txt index f1fe09fb46..4acbd21997 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt index fedc50de1e..61463feb76 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#20) Input [2]: [ca_address_sk#20, ca_state#21] Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#20, ca_state#22] (33) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#22] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#23) Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt index f1fe09fb46..4acbd21997 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/explain.txt index 58c6701f56..5c2b8c1ef6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (31) +- * Project (30) +- Window (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometSort (27) +- CometColumnarExchange (26) +- * HashAggregate (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometColumnarExchange (23) +- * HashAggregate (22) +- * Expand (21) @@ -21,12 +21,12 @@ TakeOrderedAndProject (31) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometProject (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -40,23 +40,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (4) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] @@ -75,20 +75,20 @@ Condition : isnotnull(i_item_sk#8) Input [3]: [i_item_sk#8, i_class#9, i_category#10] Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#8, i_class#11, i_category#12] (11) BroadcastExchange Input [3]: [i_item_sk#8, i_class#11, i_category#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] @@ -107,28 +107,28 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#13, s_state#14] Arguments: [s_store_sk#13], [s_store_sk#13] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [s_store_sk#13] (18) BroadcastExchange Input [1]: [s_store_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] -(21) Expand [codegen id : 4] +(21) Expand [codegen id : 1] Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 1] Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -139,10 +139,10 @@ Results [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] Arguments: hashpartitioning(i_category#15, i_class#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 5] +(24) CometNativeColumnarToRow Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 2] Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] @@ -157,14 +157,14 @@ Arguments: hashpartitioning(_w1#27, _w2#28, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] Arguments: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28], [_w1#27 ASC NULLS FIRST, _w2#28 ASC NULLS FIRST, _w0#26 ASC NULLS FIRST] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] (29) Window Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] Arguments: [rank(_w0#26) windowspecdefinition(_w1#27, _w2#28, _w0#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#29], [_w1#27, _w2#28], [_w0#26 ASC NULLS FIRST] -(30) Project [codegen id : 7] +(30) Project [codegen id : 3] Output [5]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] Input [8]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28, rank_within_parent#29] @@ -176,7 +176,7 @@ Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#30) AND (d_year#30 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#7, d_year#30] Arguments: [d_date_sk#7], [d_date_sk#7] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt index bd826eb3a3..aa40ed68cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -20,22 +20,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/simplified.txt index d6f8548c89..a6427d7111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/simplified.txt @@ -1,54 +1,46 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) + WholeStageCodegen (3) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] + Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/explain.txt index ae1d4c242b..401c5b564e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/explain.txt @@ -172,7 +172,7 @@ Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -193,7 +193,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt index 89c523a388..2bf9a90726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/simplified.txt index 61782e3011..248d0bc6e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt index ae1d4c242b..401c5b564e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt @@ -172,7 +172,7 @@ Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -193,7 +193,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt index 89c523a388..2bf9a90726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 61782e3011..248d0bc6e8 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 @@ -22,12 +22,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/explain.txt index e36c995169..17981c8b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (26) +CometNativeColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometColumnarExchange (23) @@ -11,7 +11,7 @@ : +- * BroadcastHashJoin Inner BuildRight (13) : :- * Project (11) : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -21,7 +21,7 @@ : : +- * ColumnarToRow (6) : : +- Scan parquet spark_catalog.default.inventory (5) : +- ReusedExchange (12) - +- * CometColumnarToRow (19) + +- CometNativeColumnarToRow (19) +- CometProject (18) +- CometFilter (17) +- CometNativeScan parquet spark_catalog.default.catalog_sales (16) @@ -42,7 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] (5) Scan parquet spark_catalog.default.inventory @@ -68,26 +68,26 @@ Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] Input [2]: [inv_item_sk#7, inv_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [inv_item_sk#7] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] (12) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#11] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] @@ -110,20 +110,20 @@ Condition : isnotnull(cs_item_sk#12) Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] Arguments: [cs_item_sk#12], [cs_item_sk#12] -(19) CometColumnarToRow +(19) CometNativeColumnarToRow Input [1]: [cs_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#12] Join type: Inner Join condition: None -(21) Project [codegen id : 4] +(21) Project [codegen id : 3] Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#12] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 3] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Functions: [] @@ -143,14 +143,14 @@ Functions: [] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] -(26) CometColumnarToRow [codegen id : 5] +(26) CometNativeColumnarToRow Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.date_dim (27) @@ -171,7 +171,7 @@ Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#14] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt index fb7dbb8d72..296b2c8117 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -10,7 +10,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -21,16 +21,16 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_sales diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/simplified.txt index 88a3895990..3f16bc9b50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/simplified.txt @@ -1,45 +1,41 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,cs_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [inv_item_sk,inv_date_sk] + Filter [inv_quantity_on_hand,inv_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + CometNativeColumnarToRow + CometProject [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/explain.txt index e4cfdbf6d1..c550eec11c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt index abe0937d1d..be3d6196b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/simplified.txt index 7ab646a628..f5ddbb2d04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/explain.txt index e4cfdbf6d1..c550eec11c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/extended.txt index abe0937d1d..be3d6196b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt index 7ab646a628..f5ddbb2d04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/explain.txt index f1a8620998..4d03a1f573 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (49) +CometNativeColumnarToRow (49) +- CometHashAggregate (48) +- CometExchange (47) +- CometHashAggregate (46) @@ -18,7 +18,7 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer (7) @@ -58,23 +58,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) (4) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#4, d_date#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#2] Right keys [1]: [d_date_sk#4] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, d_date#5] Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] @@ -93,24 +93,24 @@ Condition : isnotnull(c_customer_sk#6) Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#7, 20, true, false, true) AS c_first_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#8, 30, true, false, true) AS c_last_name#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] (11) BroadcastExchange Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [c_last_name#10, c_first_name#9, d_date#5] Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [c_last_name#10, c_first_name#9, d_date#5] Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] Functions: [] @@ -134,40 +134,40 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Condition : isnotnull(cs_bill_customer_sk#11) (20) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#13, d_date#14] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [2]: [cs_bill_customer_sk#11, d_date#14] Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] (23) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_bill_customer_sk#11] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [3]: [c_last_name#17, c_first_name#16, d_date#14] Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] @@ -200,40 +200,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 9] +(32) ColumnarToRow [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(33) Filter [codegen id : 9] +(33) Filter [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Condition : isnotnull(ws_bill_customer_sk#18) (34) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#20, d_date#21] -(35) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(36) Project [codegen id : 9] +(36) Project [codegen id : 3] Output [2]: [ws_bill_customer_sk#18, d_date#21] Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] (37) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(38) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_bill_customer_sk#18] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(39) Project [codegen id : 3] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 3] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] @@ -275,14 +275,14 @@ Input [1]: [count#25] Keys: [] Functions [1]: [count(1)] -(49) CometColumnarToRow [codegen id : 10] +(49) CometNativeColumnarToRow Input [1]: [count(1)#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometNativeScan parquet spark_catalog.default.date_dim (50) @@ -303,7 +303,7 @@ Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_mo Input [3]: [d_date_sk#4, d_date#5, d_month_seq#27] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt index 061475900e..6d9f5c7a17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -17,17 +17,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -44,12 +44,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer @@ -66,12 +66,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/simplified.txt index 547965d09c..f127e42be4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/simplified.txt @@ -1,75 +1,69 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] +CometNativeColumnarToRow + CometHashAggregate [count] [count(1),count(1)] + CometExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (1) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #5 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange [c_last_name,c_first_name,d_date] #5 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + CometBroadcastExchange [c_last_name,c_first_name,d_date] #7 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/explain.txt index 591fa114fb..1ca0f51516 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/explain.txt @@ -273,7 +273,7 @@ Input [1]: [count(1)#29] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -294,7 +294,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt index 55af64efc6..715ff39db7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/simplified.txt index 7ceccf5d2c..6c90e8467d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt index 591fa114fb..1ca0f51516 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt @@ -273,7 +273,7 @@ Input [1]: [count(1)#29] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -294,7 +294,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/extended.txt index 55af64efc6..715ff39db7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 7ceccf5d2c..6c90e8467d 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/explain.txt index b8ed3a361b..89eac3d545 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- * BroadcastHashJoin Inner BuildRight (44) :- * Project (24) : +- * Filter (23) : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometColumnarExchange (20) : +- * HashAggregate (19) : +- * Project (18) @@ -19,11 +19,11 @@ : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.inventory (1) : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) + : : : +- CometNativeColumnarToRow (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) + : : +- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : +- ReusedExchange (16) @@ -31,7 +31,7 @@ +- * Project (42) +- * Filter (41) +- * HashAggregate (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometColumnarExchange (38) +- * HashAggregate (37) +- * Project (36) @@ -56,10 +56,10 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) @@ -74,20 +74,20 @@ ReadSchema: struct Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [i_item_sk#6] (7) BroadcastExchange Input [1]: [i_item_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] @@ -102,37 +102,37 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] (13) BroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] (16) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#9, d_moy#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] @@ -143,21 +143,21 @@ Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometColumnarToRow [codegen id : 10] +(21) CometNativeColumnarToRow Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(22) HashAggregate [codegen id : 10] +(22) HashAggregate [codegen id : 4] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(23) Filter [codegen id : 10] +(23) Filter [codegen id : 4] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END -(24) Project [codegen id : 10] +(24) Project [codegen id : 4] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -169,53 +169,53 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 8] +(26) ColumnarToRow [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(27) Filter [codegen id : 8] +(27) Filter [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) (28) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(29) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(30) Project [codegen id : 8] +(30) Project [codegen id : 2] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] (31) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] (34) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#34, d_moy#35] -(35) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(36) Project [codegen id : 8] +(36) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(37) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 2] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] @@ -226,21 +226,21 @@ Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n# Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) CometColumnarToRow [codegen id : 9] +(39) CometNativeColumnarToRow Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 3] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] -(41) Filter [codegen id : 9] +(41) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END -(42) Project [codegen id : 9] +(42) Project [codegen id : 3] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] @@ -248,7 +248,7 @@ Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(44) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner @@ -262,14 +262,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 11] +(47) CometNativeColumnarToRow Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometNativeScan parquet spark_catalog.default.date_dim (48) @@ -290,7 +290,7 @@ Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2 Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#10] (52) BroadcastExchange @@ -299,7 +299,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -320,7 +320,7 @@ Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2 Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#34, d_moy#35] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt index e44cca98a1..d9cec17b28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- BroadcastHashJoin :- Project : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,20 +19,20 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,7 +40,7 @@ CometColumnarToRow +- Project +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -54,20 +54,20 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/simplified.txt index 11dd59a97f..ff7f9077d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/simplified.txt @@ -1,84 +1,74 @@ -WholeStageCodegen (11) - 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 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + 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 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (1) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (2) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk] #4 + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + ReusedExchange [i_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/explain.txt index ed74780c2c..fce6b102d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/explain.txt @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/simplified.txt index 2cd1f70c8a..0c4aa97a2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 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 ed74780c2c..fce6b102d0 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 @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 2cd1f70c8a..0c4aa97a2b 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/explain.txt index 69fbdb52f0..4510aa3b77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- * BroadcastHashJoin Inner BuildRight (44) :- * Project (24) : +- * Filter (23) : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometColumnarExchange (20) : +- * HashAggregate (19) : +- * Project (18) @@ -19,11 +19,11 @@ : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.inventory (1) : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) + : : : +- CometNativeColumnarToRow (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) + : : +- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : +- ReusedExchange (16) @@ -31,7 +31,7 @@ +- * Project (42) +- * Filter (41) +- * HashAggregate (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometColumnarExchange (38) +- * HashAggregate (37) +- * Project (36) @@ -56,10 +56,10 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) @@ -74,20 +74,20 @@ ReadSchema: struct Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [i_item_sk#6] (7) BroadcastExchange Input [1]: [i_item_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] @@ -102,37 +102,37 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] (13) BroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] (16) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#9, d_moy#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] @@ -143,21 +143,21 @@ Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometColumnarToRow [codegen id : 10] +(21) CometNativeColumnarToRow Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(22) HashAggregate [codegen id : 10] +(22) HashAggregate [codegen id : 4] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(23) Filter [codegen id : 10] +(23) Filter [codegen id : 4] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.5) END) -(24) Project [codegen id : 10] +(24) Project [codegen id : 4] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -169,53 +169,53 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 8] +(26) ColumnarToRow [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(27) Filter [codegen id : 8] +(27) Filter [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) (28) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(29) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(30) Project [codegen id : 8] +(30) Project [codegen id : 2] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] (31) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] (34) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#34, d_moy#35] -(35) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(36) Project [codegen id : 8] +(36) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(37) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 2] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] @@ -226,21 +226,21 @@ Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n# Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) CometColumnarToRow [codegen id : 9] +(39) CometNativeColumnarToRow Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 3] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] -(41) Filter [codegen id : 9] +(41) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END -(42) Project [codegen id : 9] +(42) Project [codegen id : 3] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] @@ -248,7 +248,7 @@ Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(44) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner @@ -262,14 +262,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 11] +(47) CometNativeColumnarToRow Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometNativeScan parquet spark_catalog.default.date_dim (48) @@ -290,7 +290,7 @@ Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2 Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#10] (52) BroadcastExchange @@ -299,7 +299,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -320,7 +320,7 @@ Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2 Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#34, d_moy#35] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt index e44cca98a1..d9cec17b28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- BroadcastHashJoin :- Project : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,20 +19,20 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,7 +40,7 @@ CometColumnarToRow +- Project +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -54,20 +54,20 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/simplified.txt index 11dd59a97f..ff7f9077d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/simplified.txt @@ -1,84 +1,74 @@ -WholeStageCodegen (11) - 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 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + 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 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (1) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (2) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk] #4 + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + ReusedExchange [i_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/explain.txt index 2a31b0d8b2..b2934e5cae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/explain.txt @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/simplified.txt index 2cd1f70c8a..0c4aa97a2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 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 2a31b0d8b2..b2934e5cae 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 @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 2cd1f70c8a..0c4aa97a2b 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/explain.txt index 9fffd1dde5..926367ac8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/explain.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject (120) : : : :- * BroadcastHashJoin Inner BuildRight (37) : : : : :- * Filter (18) : : : : : +- * HashAggregate (17) - : : : : : +- * CometColumnarToRow (16) + : : : : : +- CometNativeColumnarToRow (16) : : : : : +- CometColumnarExchange (15) : : : : : +- * HashAggregate (14) : : : : : +- * Project (13) : : : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : : : :- * Project (10) : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * CometColumnarToRow (4) + : : : : : : :- CometNativeColumnarToRow (4) : : : : : : : +- CometProject (3) : : : : : : : +- CometFilter (2) : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -29,14 +29,14 @@ TakeOrderedAndProject (120) : : : : : +- ReusedExchange (11) : : : : +- BroadcastExchange (36) : : : : +- * HashAggregate (35) - : : : : +- * CometColumnarToRow (34) + : : : : +- CometNativeColumnarToRow (34) : : : : +- CometColumnarExchange (33) : : : : +- * HashAggregate (32) : : : : +- * Project (31) : : : : +- * BroadcastHashJoin Inner BuildRight (30) : : : : :- * Project (28) : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * CometColumnarToRow (22) + : : : : : :- CometNativeColumnarToRow (22) : : : : : : +- CometProject (21) : : : : : : +- CometFilter (20) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -48,14 +48,14 @@ TakeOrderedAndProject (120) : : : +- BroadcastExchange (56) : : : +- * Filter (55) : : : +- * HashAggregate (54) - : : : +- * CometColumnarToRow (53) + : : : +- CometNativeColumnarToRow (53) : : : +- CometColumnarExchange (52) : : : +- * HashAggregate (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) : : : :- * Project (47) : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * CometColumnarToRow (41) + : : : : :- CometNativeColumnarToRow (41) : : : : : +- CometProject (40) : : : : : +- CometFilter (39) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -66,14 +66,14 @@ TakeOrderedAndProject (120) : : : +- ReusedExchange (48) : : +- BroadcastExchange (76) : : +- * HashAggregate (75) - : : +- * CometColumnarToRow (74) + : : +- CometNativeColumnarToRow (74) : : +- CometColumnarExchange (73) : : +- * HashAggregate (72) : : +- * Project (71) : : +- * BroadcastHashJoin Inner BuildRight (70) : : :- * Project (68) : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : :- * CometColumnarToRow (62) + : : : :- CometNativeColumnarToRow (62) : : : : +- CometProject (61) : : : : +- CometFilter (60) : : : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -85,14 +85,14 @@ TakeOrderedAndProject (120) : +- BroadcastExchange (97) : +- * Filter (96) : +- * HashAggregate (95) - : +- * CometColumnarToRow (94) + : +- CometNativeColumnarToRow (94) : +- CometColumnarExchange (93) : +- * HashAggregate (92) : +- * Project (91) : +- * BroadcastHashJoin Inner BuildRight (90) : :- * Project (88) : : +- * BroadcastHashJoin Inner BuildRight (87) - : : :- * CometColumnarToRow (82) + : : :- CometNativeColumnarToRow (82) : : : +- CometProject (81) : : : +- CometFilter (80) : : : +- CometNativeScan parquet spark_catalog.default.customer (79) @@ -103,14 +103,14 @@ TakeOrderedAndProject (120) : +- ReusedExchange (89) +- BroadcastExchange (117) +- * HashAggregate (116) - +- * CometColumnarToRow (115) + +- CometNativeColumnarToRow (115) +- CometColumnarExchange (114) +- * HashAggregate (113) +- * Project (112) +- * BroadcastHashJoin Inner BuildRight (111) :- * Project (109) : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * CometColumnarToRow (103) + : :- CometNativeColumnarToRow (103) : : +- CometProject (102) : : +- CometFilter (101) : : +- CometNativeScan parquet spark_catalog.default.customer (100) @@ -136,7 +136,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] (5) Scan parquet spark_catalog.default.store_sales @@ -158,30 +158,30 @@ Condition : isnotnull(ss_customer_sk#15) Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#15] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] (11) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#22, d_year#23] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#20] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] @@ -192,17 +192,17 @@ Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cus Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 24] +(16) CometNativeColumnarToRow Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] -(17) HashAggregate [codegen id : 24] +(17) HashAggregate [codegen id : 18] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] Aggregate Attributes [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28] Results [2]: [c_customer_id#9 AS customer_id#29, sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28 AS year_total#30] -(18) Filter [codegen id : 24] +(18) Filter [codegen id : 18] Input [2]: [customer_id#29, year_total#30] Condition : (isnotnull(year_total#30) AND (year_total#30 > 0.000000)) @@ -221,7 +221,7 @@ Condition : (isnotnull(c_customer_sk#31) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] Arguments: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14], [c_customer_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#32, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#33, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#34, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#35, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#37, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#38, 50, true, false, true) AS c_email_address#14] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [8]: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14] (23) Scan parquet spark_catalog.default.store_sales @@ -232,10 +232,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] Condition : isnotnull(ss_customer_sk#39) @@ -243,30 +243,30 @@ Condition : isnotnull(ss_customer_sk#39) Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#31] Right keys [1]: [ss_customer_sk#39] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] Input [14]: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] (29) ReusedExchange [Reuses operator id: 128] Output [2]: [d_date_sk#46, d_year#47] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#44] Right keys [1]: [d_date_sk#46] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, d_year#47] Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, d_year#47] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47] Functions [1]: [partial_sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))] @@ -277,10 +277,10 @@ Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cus Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47] Functions [1]: [sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))] @@ -291,7 +291,7 @@ Results [8]: [c_customer_id#9 AS customer_id#52, c_first_name#10 AS customer_fir Input [8]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 24] +(37) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#52] Join type: Inner @@ -312,7 +312,7 @@ Condition : (isnotnull(c_customer_sk#60) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] Arguments: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73], [c_customer_sk#60, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#61, 16, true, false, true) AS c_customer_id#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#62, 20, true, false, true) AS c_first_name#69, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#63, 30, true, false, true) AS c_last_name#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#64, 1, true, false, true) AS c_preferred_cust_flag#71, c_birth_country#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#66, 13, true, false, true) AS c_login#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#67, 50, true, false, true) AS c_email_address#73] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [8]: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73] (42) Scan parquet spark_catalog.default.catalog_sales @@ -323,10 +323,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#79), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] Condition : isnotnull(cs_bill_customer_sk#74) @@ -334,30 +334,30 @@ Condition : isnotnull(cs_bill_customer_sk#74) Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#60] Right keys [1]: [cs_bill_customer_sk#74] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] Input [14]: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] (48) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#80, d_year#81] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#79] Right keys [1]: [d_date_sk#80] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, d_year#81] Input [14]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79, d_date_sk#80, d_year#81] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, d_year#81] Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81] Functions [1]: [partial_sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))] @@ -368,17 +368,17 @@ Results [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cu Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81] Functions [1]: [sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))] Aggregate Attributes [1]: [sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))#86] Results [2]: [c_customer_id#68 AS customer_id#87, sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))#86 AS year_total#88] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#87, year_total#88] Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) @@ -386,13 +386,13 @@ Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) Input [2]: [customer_id#87, year_total#88] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 24] +(57) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#87] Join type: Inner Join condition: None -(58) Project [codegen id : 24] +(58) Project [codegen id : 18] Output [11]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#88] Input [12]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, customer_id#87, year_total#88] @@ -411,7 +411,7 @@ Condition : (isnotnull(c_customer_sk#89) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] Arguments: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73], [c_customer_sk#89, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#90, 16, true, false, true) AS c_customer_id#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#91, 20, true, false, true) AS c_first_name#69, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#92, 30, true, false, true) AS c_last_name#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#93, 1, true, false, true) AS c_preferred_cust_flag#71, c_birth_country#94, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#95, 13, true, false, true) AS c_login#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#96, 50, true, false, true) AS c_email_address#73] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [8]: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73] (63) Scan parquet spark_catalog.default.catalog_sales @@ -422,10 +422,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#102), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] Condition : isnotnull(cs_bill_customer_sk#97) @@ -433,30 +433,30 @@ Condition : isnotnull(cs_bill_customer_sk#97) Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#89] Right keys [1]: [cs_bill_customer_sk#97] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] Input [14]: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] (69) ReusedExchange [Reuses operator id: 128] Output [2]: [d_date_sk#103, d_year#104] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#102] Right keys [1]: [d_date_sk#103] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#104] Input [14]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102, d_date_sk#103, d_year#104] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#104] Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104] Functions [1]: [partial_sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))] @@ -467,10 +467,10 @@ Results [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cu Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104] Functions [1]: [sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))] @@ -481,13 +481,13 @@ Results [2]: [c_customer_id#68 AS customer_id#109, sum(((((cs_ext_list_price#101 Input [2]: [customer_id#109, year_total#110] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 24] +(77) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#109] Join type: Inner Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN (year_total#110 / year_total#88) END > CASE WHEN (year_total#30 > 0.000000) THEN (year_total#59 / year_total#30) END) -(78) Project [codegen id : 24] +(78) Project [codegen id : 18] Output [10]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110] Input [13]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#88, customer_id#109, year_total#110] @@ -506,7 +506,7 @@ Condition : (isnotnull(c_customer_sk#111) AND isnotnull(staticinvoke(class org.a Input [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] Arguments: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124], [c_customer_sk#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#112, 16, true, false, true) AS c_customer_id#119, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#113, 20, true, false, true) AS c_first_name#120, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#114, 30, true, false, true) AS c_last_name#121, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#115, 1, true, false, true) AS c_preferred_cust_flag#122, c_birth_country#116, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#117, 13, true, false, true) AS c_login#123, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#118, 50, true, false, true) AS c_email_address#124] -(82) CometColumnarToRow [codegen id : 18] +(82) CometNativeColumnarToRow Input [8]: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124] (83) Scan parquet spark_catalog.default.web_sales @@ -517,10 +517,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#130), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(84) ColumnarToRow [codegen id : 16] +(84) ColumnarToRow [codegen id : 12] Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -(85) Filter [codegen id : 16] +(85) Filter [codegen id : 12] Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] Condition : isnotnull(ws_bill_customer_sk#125) @@ -528,30 +528,30 @@ Condition : isnotnull(ws_bill_customer_sk#125) Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(87) BroadcastHashJoin [codegen id : 18] +(87) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_customer_sk#111] Right keys [1]: [ws_bill_customer_sk#125] Join type: Inner Join condition: None -(88) Project [codegen id : 18] +(88) Project [codegen id : 13] Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] Input [14]: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] (89) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#131, d_year#132] -(90) BroadcastHashJoin [codegen id : 18] +(90) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#130] Right keys [1]: [d_date_sk#131] Join type: Inner Join condition: None -(91) Project [codegen id : 18] +(91) Project [codegen id : 13] Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, d_year#132] Input [14]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130, d_date_sk#131, d_year#132] -(92) HashAggregate [codegen id : 18] +(92) HashAggregate [codegen id : 13] Input [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, d_year#132] Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132] Functions [1]: [partial_sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))] @@ -562,17 +562,17 @@ Results [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] Arguments: hashpartitioning(c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(94) CometColumnarToRow [codegen id : 19] +(94) CometNativeColumnarToRow Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] -(95) HashAggregate [codegen id : 19] +(95) HashAggregate [codegen id : 14] Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132] Functions [1]: [sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))] Aggregate Attributes [1]: [sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))#137] Results [2]: [c_customer_id#119 AS customer_id#138, sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))#137 AS year_total#139] -(96) Filter [codegen id : 19] +(96) Filter [codegen id : 14] Input [2]: [customer_id#138, year_total#139] Condition : (isnotnull(year_total#139) AND (year_total#139 > 0.000000)) @@ -580,13 +580,13 @@ Condition : (isnotnull(year_total#139) AND (year_total#139 > 0.000000)) Input [2]: [customer_id#138, year_total#139] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] -(98) BroadcastHashJoin [codegen id : 24] +(98) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#138] Join type: Inner Join condition: None -(99) Project [codegen id : 24] +(99) Project [codegen id : 18] Output [11]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, year_total#139] Input [12]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, customer_id#138, year_total#139] @@ -605,7 +605,7 @@ Condition : (isnotnull(c_customer_sk#140) AND isnotnull(staticinvoke(class org.a Input [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] Arguments: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124], [c_customer_sk#140, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#141, 16, true, false, true) AS c_customer_id#119, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#142, 20, true, false, true) AS c_first_name#120, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#143, 30, true, false, true) AS c_last_name#121, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#144, 1, true, false, true) AS c_preferred_cust_flag#122, c_birth_country#145, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#146, 13, true, false, true) AS c_login#123, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#147, 50, true, false, true) AS c_email_address#124] -(103) CometColumnarToRow [codegen id : 22] +(103) CometNativeColumnarToRow Input [8]: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124] (104) Scan parquet spark_catalog.default.web_sales @@ -616,10 +616,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 20] +(105) ColumnarToRow [codegen id : 15] Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -(106) Filter [codegen id : 20] +(106) Filter [codegen id : 15] Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] Condition : isnotnull(ws_bill_customer_sk#148) @@ -627,30 +627,30 @@ Condition : isnotnull(ws_bill_customer_sk#148) Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -(108) BroadcastHashJoin [codegen id : 22] +(108) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_customer_sk#140] Right keys [1]: [ws_bill_customer_sk#148] Join type: Inner Join condition: None -(109) Project [codegen id : 22] +(109) Project [codegen id : 16] Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] Input [14]: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] (110) ReusedExchange [Reuses operator id: 128] Output [2]: [d_date_sk#154, d_year#155] -(111) BroadcastHashJoin [codegen id : 22] +(111) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_sold_date_sk#153] Right keys [1]: [d_date_sk#154] Join type: Inner Join condition: None -(112) Project [codegen id : 22] +(112) Project [codegen id : 16] Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#155] Input [14]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153, d_date_sk#154, d_year#155] -(113) HashAggregate [codegen id : 22] +(113) HashAggregate [codegen id : 16] Input [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#155] Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155] Functions [1]: [partial_sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))] @@ -661,10 +661,10 @@ Results [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] Arguments: hashpartitioning(c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(115) CometColumnarToRow [codegen id : 23] +(115) CometNativeColumnarToRow Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] -(116) HashAggregate [codegen id : 23] +(116) HashAggregate [codegen id : 17] Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155] Functions [1]: [sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))] @@ -675,13 +675,13 @@ Results [2]: [c_customer_id#119 AS customer_id#160, sum(((((ws_ext_list_price#15 Input [2]: [customer_id#160, year_total#161] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] -(118) BroadcastHashJoin [codegen id : 24] +(118) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#160] Join type: Inner Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN (year_total#110 / year_total#88) END > CASE WHEN (year_total#139 > 0.000000) THEN (year_total#161 / year_total#139) END) -(119) Project [codegen id : 24] +(119) Project [codegen id : 18] Output [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] Input [13]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, year_total#139, customer_id#160, year_total#161] @@ -693,7 +693,7 @@ Arguments: 100, [customer_id#52 ASC NULLS FIRST, customer_first_name#53 ASC NULL Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometFilter (122) +- CometNativeScan parquet spark_catalog.default.date_dim (121) @@ -709,7 +709,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [2]: [d_date_sk#22, d_year#23] (124) BroadcastExchange @@ -718,7 +718,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#45 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometNativeScan parquet spark_catalog.default.date_dim (125) @@ -734,7 +734,7 @@ ReadSchema: struct Input [2]: [d_date_sk#46, d_year#47] Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#46)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#46, d_year#47] (128) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt index 54e1a1c11a..737217ff91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt @@ -10,14 +10,14 @@ TakeOrderedAndProject : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- HashAggregate - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- HashAggregate : : : : : +- Project : : : : : +- BroadcastHashJoin : : : : : :- Project : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -27,23 +27,23 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- Project : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -53,24 +53,24 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -80,19 +80,19 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -102,20 +102,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -125,19 +125,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -147,7 +147,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/simplified.txt index 56bc32e144..dd6fd6a2ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - WholeStageCodegen (24) + WholeStageCodegen (18) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] @@ -11,17 +11,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] @@ -34,69 +34,65 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter @@ -106,25 +102,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter @@ -134,26 +130,26 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #14 - WholeStageCodegen (19) + WholeStageCodegen (14) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) + WholeStageCodegen (13) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #16 - WholeStageCodegen (16) + WholeStageCodegen (12) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -163,25 +159,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #17 - WholeStageCodegen (23) + WholeStageCodegen (17) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) + WholeStageCodegen (16) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #19 - WholeStageCodegen (20) + WholeStageCodegen (15) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/explain.txt index cea1c1d430..979d7e593c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/explain.txt @@ -612,7 +612,7 @@ Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (111) -+- * CometColumnarToRow (110) ++- CometNativeColumnarToRow (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -628,7 +628,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(110) CometColumnarToRow [codegen id : 1] +(110) CometNativeColumnarToRow Input [2]: [d_date_sk#22, d_year#23] (111) BroadcastExchange @@ -637,7 +637,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 BroadcastExchange (115) -+- * CometColumnarToRow (114) ++- CometNativeColumnarToRow (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -653,7 +653,7 @@ ReadSchema: struct Input [2]: [d_date_sk#43, d_year#44] Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) -(114) CometColumnarToRow [codegen id : 1] +(114) CometNativeColumnarToRow Input [2]: [d_date_sk#43, d_year#44] (115) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt index 1be0816ee1..2336733a6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -47,7 +47,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/simplified.txt index 36be965b48..f984ee0702 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/simplified.txt @@ -27,11 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -51,11 +49,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt index cea1c1d430..979d7e593c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt @@ -612,7 +612,7 @@ Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (111) -+- * CometColumnarToRow (110) ++- CometNativeColumnarToRow (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -628,7 +628,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(110) CometColumnarToRow [codegen id : 1] +(110) CometNativeColumnarToRow Input [2]: [d_date_sk#22, d_year#23] (111) BroadcastExchange @@ -637,7 +637,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 BroadcastExchange (115) -+- * CometColumnarToRow (114) ++- CometNativeColumnarToRow (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -653,7 +653,7 @@ ReadSchema: struct Input [2]: [d_date_sk#43, d_year#44] Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) -(114) CometColumnarToRow [codegen id : 1] +(114) CometNativeColumnarToRow Input [2]: [d_date_sk#43, d_year#44] (115) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/extended.txt index 1be0816ee1..2336733a6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -47,7 +47,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange 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 36be965b48..f984ee0702 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 @@ -27,11 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -51,11 +49,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/explain.txt index fe074b9df2..43157ba891 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (34) +CometNativeColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -188,14 +188,14 @@ Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -(34) CometColumnarToRow [codegen id : 2] +(34) CometNativeColumnarToRow Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometFilter (36) +- CometNativeScan parquet spark_catalog.default.date_dim (35) @@ -211,7 +211,7 @@ ReadSchema: struct Input [2]: [d_date_sk#18, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [2]: [d_date_sk#18, d_date#19] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt index d5dbd76871..f1d98d3059 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -18,7 +18,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/simplified.txt index 13760fcb0b..e28fe12099 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/simplified.txt @@ -1,45 +1,41 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - 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 [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] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #2 - WholeStageCodegen (1) - Filter [cs_warehouse_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] + 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 [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] + CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] + CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] + CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] + CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #2 + WholeStageCodegen (1) + Filter [cs_warehouse_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] + CometExchange [cr_order_number,cr_item_sk] #4 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_state] #5 + CometProject [w_state] [w_warehouse_sk,w_state] + CometFilter [w_warehouse_sk,w_state] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/explain.txt index 00e2b8fbc1..7057990127 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -207,7 +207,7 @@ ReadSchema: struct Input [2]: [d_date_sk#18, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [2]: [d_date_sk#18, d_date#19] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt index 54a7b21486..8d2e593975 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/simplified.txt index 79e3556750..96423b87fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/simplified.txt @@ -19,11 +19,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] CometExchange [cr_order_number,cr_item_sk] #4 CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt index 00e2b8fbc1..7057990127 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt @@ -191,7 +191,7 @@ Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -207,7 +207,7 @@ ReadSchema: struct Input [2]: [d_date_sk#18, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [2]: [d_date_sk#18, d_date#19] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/extended.txt index 54a7b21486..8d2e593975 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort 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 79e3556750..96423b87fb 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 @@ -19,11 +19,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] CometExchange [cr_order_number,cr_item_sk] #4 CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/explain.txt index 625df9cd57..cd89e72e3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -102,6 +102,6 @@ Functions: [] Input [1]: [i_product_name#12] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [1]: [i_product_name#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/extended.txt index b40ebddb6f..6f7d077cf3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/simplified.txt index 2557f51684..76e150734c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometExchange [i_product_name] #1 CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + CometProject [i_product_name] [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometNativeScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] + CometExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/explain.txt index 568da58fa0..96d91f76b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -102,6 +102,6 @@ Functions: [] Input [1]: [i_product_name#12] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [1]: [i_product_name#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/extended.txt index 9d802b5033..a3c8dd9568 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/simplified.txt index 99c935aaa4..586ccd2852 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometExchange [i_product_name] #1 CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + CometProject [i_product_name] [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] + CometExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt index 568da58fa0..96d91f76b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -102,6 +102,6 @@ Functions: [] Input [1]: [i_product_name#12] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [1]: [i_product_name#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/extended.txt index 9d802b5033..a3c8dd9568 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange 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 99c935aaa4..586ccd2852 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometExchange [i_product_name] #1 CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + CometProject [i_product_name] [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] + CometExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/explain.txt index af01446009..4c0c21880d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/simplified.txt index 96ab34454d..6ad3968250 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(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 [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] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(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 [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] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 + CometProject [i_category] [i_item_sk,i_category_id,i_category] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/explain.txt index 261e6ddbaa..4172904405 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_s Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/extended.txt index 5eaea983af..cebd2e28d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/simplified.txt index c7064b3ad8..d40373ef42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -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 [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 [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] - CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] +CometNativeColumnarToRow + CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + 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 [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] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [d_day_name] [d_date_sk,d_day_name] + CometFilter [d_date_sk,d_year,d_day_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/explain.txt index b2a660d9d6..581323476b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/explain.txt @@ -11,11 +11,11 @@ TakeOrderedAndProject (45) : : : +- * Filter (16) : : : +- Window (15) : : : +- WindowGroupLimit (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometSort (12) : : : +- CometColumnarExchange (11) : : : +- WindowGroupLimit (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometSort (8) : : : +- CometFilter (7) : : : +- CometHashAggregate (6) @@ -29,17 +29,17 @@ TakeOrderedAndProject (45) : : +- * Filter (30) : : +- Window (29) : : +- WindowGroupLimit (28) - : : +- * CometColumnarToRow (27) + : : +- CometNativeColumnarToRow (27) : : +- CometSort (26) : : +- CometColumnarExchange (25) : : +- WindowGroupLimit (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometSort (22) : : +- CometFilter (21) : : +- CometHashAggregate (20) : : +- ReusedExchange (19) : +- BroadcastExchange (39) - : +- * CometColumnarToRow (38) + : +- CometNativeColumnarToRow (38) : +- CometProject (37) : +- CometFilter (36) : +- CometNativeScan parquet spark_catalog.default.item (35) @@ -83,7 +83,7 @@ Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(9) CometColumnarToRow [codegen id : 1] +(9) CometNativeColumnarToRow Input [2]: [item_sk#7, rank_col#8] (10) WindowGroupLimit @@ -98,7 +98,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [2]: [item_sk#7, rank_col#8] (14) WindowGroupLimit @@ -109,15 +109,15 @@ Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final Input [2]: [item_sk#7, rank_col#8] Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] -(16) Filter [codegen id : 3] +(16) Filter [codegen id : 1] Input [3]: [item_sk#7, rank_col#8, rnk#11] Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(17) Project [codegen id : 3] +(17) Project [codegen id : 1] Output [2]: [item_sk#7, rnk#11] Input [3]: [item_sk#7, rank_col#8, rnk#11] -(18) Sort [codegen id : 3] +(18) Sort [codegen id : 1] Input [2]: [item_sk#7, rnk#11] Arguments: [rnk#11 ASC NULLS FIRST], false, 0 @@ -137,7 +137,7 @@ Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0 Input [2]: [item_sk#16, rank_col#17] Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [item_sk#16, rank_col#17] (24) WindowGroupLimit @@ -152,7 +152,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [2]: [item_sk#16, rank_col#17] Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] -(27) CometColumnarToRow [codegen id : 5] +(27) CometNativeColumnarToRow Input [2]: [item_sk#16, rank_col#17] (28) WindowGroupLimit @@ -163,25 +163,25 @@ Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final Input [2]: [item_sk#16, rank_col#17] Arguments: [rank(rank_col#17) windowspecdefinition(rank_col#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#18], [rank_col#17 DESC NULLS LAST] -(30) Filter [codegen id : 6] +(30) Filter [codegen id : 2] Input [3]: [item_sk#16, rank_col#17, rnk#18] Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [2]: [item_sk#16, rnk#18] Input [3]: [item_sk#16, rank_col#17, rnk#18] -(32) Sort [codegen id : 6] +(32) Sort [codegen id : 2] Input [2]: [item_sk#16, rnk#18] Arguments: [rnk#18 ASC NULLS FIRST], false, 0 -(33) SortMergeJoin [codegen id : 9] +(33) SortMergeJoin [codegen id : 3] Left keys [1]: [rnk#11] Right keys [1]: [rnk#18] Join type: Inner Join condition: None -(34) Project [codegen id : 9] +(34) Project [codegen id : 3] Output [3]: [item_sk#7, rnk#11, item_sk#16] Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] @@ -200,33 +200,33 @@ Condition : isnotnull(i_item_sk#19) Input [2]: [i_item_sk#19, i_product_name#20] Arguments: [i_item_sk#19, i_product_name#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#20, 50, true, false, true) AS i_product_name#21] -(38) CometColumnarToRow [codegen id : 7] +(38) CometNativeColumnarToRow Input [2]: [i_item_sk#19, i_product_name#21] (39) BroadcastExchange Input [2]: [i_item_sk#19, i_product_name#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#19] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(41) Project [codegen id : 3] Output [3]: [rnk#11, item_sk#16, i_product_name#21] Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#21] (42) ReusedExchange [Reuses operator id: 39] Output [2]: [i_item_sk#22, i_product_name#23] -(43) BroadcastHashJoin [codegen id : 9] +(43) BroadcastHashJoin [codegen id : 3] Left keys [1]: [item_sk#16] Right keys [1]: [i_item_sk#22] Join type: Inner Join condition: None -(44) Project [codegen id : 9] +(44) Project [codegen id : 3] Output [3]: [rnk#11, i_product_name#21 AS best_performing#24, i_product_name#23 AS worst_performing#25] Input [5]: [rnk#11, item_sk#16, i_product_name#21, i_item_sk#22, i_product_name#23] @@ -237,7 +237,7 @@ Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#24, worst_per ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -275,7 +275,7 @@ Input [3]: [ss_store_sk#27, sum#30, count#31] Keys [1]: [ss_store_sk#27] Functions [1]: [avg(UnscaledValue(ss_net_profit#28))] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [rank_col#32] Subquery:2 Hosting operator id = 21 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/extended.txt index 236d51688a..3ee645114d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/extended.txt @@ -10,15 +10,15 @@ TakeOrderedAndProject : : : +- Filter : : : +- Window : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometSort : : : +- CometColumnarExchange : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometSort : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -36,11 +36,11 @@ TakeOrderedAndProject : : +- Filter : : +- Window : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometFilter : : : +- ReusedSubquery @@ -51,12 +51,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/simplified.txt index a6db5c3397..3690d038d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (9) + WholeStageCodegen (3) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -7,69 +7,57 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Project [item_sk,rnk,item_sk] SortMergeJoin [rnk,rnk] InputAdapter - WholeStageCodegen (3) + WholeStageCodegen (1) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] WindowGroupLimit [rank_col] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #1 - WindowGroupLimit [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometNativeColumnarToRow + CometSort [item_sk,rank_col] + CometColumnarExchange #1 + WindowGroupLimit [rank_col] + CometNativeColumnarToRow + CometSort [item_sk,rank_col] + CometFilter [item_sk,rank_col] + Subquery #1 + CometNativeColumnarToRow + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_store_sk] #3 + 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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_item_sk] #2 + 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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (2) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] WindowGroupLimit [rank_col] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #4 - WindowGroupLimit [rank_col] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - ReusedSubquery [rank_col] #1 - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - ReusedExchange [ss_item_sk,sum,count] #2 + CometNativeColumnarToRow + CometSort [item_sk,rank_col] + CometColumnarExchange #4 + WindowGroupLimit [rank_col] + CometNativeColumnarToRow + CometSort [item_sk,rank_col] + CometFilter [item_sk,rank_col] + ReusedSubquery [rank_col] #1 + 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 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_product_name] + CometNativeColumnarToRow + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/explain.txt index 4b30c7d51f..cd5bc2bb2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/explain.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject (45) : : +- CometHashAggregate (20) : : +- ReusedExchange (19) : +- BroadcastExchange (39) - : +- * CometColumnarToRow (38) + : +- CometNativeColumnarToRow (38) : +- CometProject (37) : +- CometFilter (36) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (35) @@ -175,13 +175,13 @@ Input [3]: [item_sk#16, rank_col#17, rnk#18] Input [2]: [item_sk#16, rnk#18] Arguments: [rnk#18 ASC NULLS FIRST], false, 0 -(33) SortMergeJoin [codegen id : 9] +(33) SortMergeJoin [codegen id : 7] Left keys [1]: [rnk#11] Right keys [1]: [rnk#18] Join type: Inner Join condition: None -(34) Project [codegen id : 9] +(34) Project [codegen id : 7] Output [3]: [item_sk#7, rnk#11, item_sk#16] Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] @@ -200,33 +200,33 @@ Condition : isnotnull(i_item_sk#19) Input [2]: [i_item_sk#19, i_product_name#20] Arguments: [i_item_sk#19, i_product_name#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#20, 50, true, false, true) AS i_product_name#21] -(38) CometColumnarToRow [codegen id : 7] +(38) CometNativeColumnarToRow Input [2]: [i_item_sk#19, i_product_name#21] (39) BroadcastExchange Input [2]: [i_item_sk#19, i_product_name#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 7] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#19] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(41) Project [codegen id : 7] Output [3]: [rnk#11, item_sk#16, i_product_name#21] Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#21] (42) ReusedExchange [Reuses operator id: 39] Output [2]: [i_item_sk#22, i_product_name#23] -(43) BroadcastHashJoin [codegen id : 9] +(43) BroadcastHashJoin [codegen id : 7] Left keys [1]: [item_sk#16] Right keys [1]: [i_item_sk#22] Join type: Inner Join condition: None -(44) Project [codegen id : 9] +(44) Project [codegen id : 7] Output [3]: [rnk#11, i_product_name#21 AS best_performing#24, i_product_name#23 AS worst_performing#25] Input [5]: [rnk#11, item_sk#16, i_product_name#21, i_item_sk#22, i_product_name#23] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt index 6b58597252..7e28f0cb5b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt @@ -51,12 +51,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/simplified.txt index d0c3ad435b..75bbcfd60a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (9) + WholeStageCodegen (7) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -65,11 +65,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ReusedExchange [ss_item_sk,sum,count] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] + CometNativeColumnarToRow + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt index 4b30c7d51f..cd5bc2bb2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject (45) : : +- CometHashAggregate (20) : : +- ReusedExchange (19) : +- BroadcastExchange (39) - : +- * CometColumnarToRow (38) + : +- CometNativeColumnarToRow (38) : +- CometProject (37) : +- CometFilter (36) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (35) @@ -175,13 +175,13 @@ Input [3]: [item_sk#16, rank_col#17, rnk#18] Input [2]: [item_sk#16, rnk#18] Arguments: [rnk#18 ASC NULLS FIRST], false, 0 -(33) SortMergeJoin [codegen id : 9] +(33) SortMergeJoin [codegen id : 7] Left keys [1]: [rnk#11] Right keys [1]: [rnk#18] Join type: Inner Join condition: None -(34) Project [codegen id : 9] +(34) Project [codegen id : 7] Output [3]: [item_sk#7, rnk#11, item_sk#16] Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] @@ -200,33 +200,33 @@ Condition : isnotnull(i_item_sk#19) Input [2]: [i_item_sk#19, i_product_name#20] Arguments: [i_item_sk#19, i_product_name#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#20, 50, true, false, true) AS i_product_name#21] -(38) CometColumnarToRow [codegen id : 7] +(38) CometNativeColumnarToRow Input [2]: [i_item_sk#19, i_product_name#21] (39) BroadcastExchange Input [2]: [i_item_sk#19, i_product_name#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 7] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#19] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(41) Project [codegen id : 7] Output [3]: [rnk#11, item_sk#16, i_product_name#21] Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#21] (42) ReusedExchange [Reuses operator id: 39] Output [2]: [i_item_sk#22, i_product_name#23] -(43) BroadcastHashJoin [codegen id : 9] +(43) BroadcastHashJoin [codegen id : 7] Left keys [1]: [item_sk#16] Right keys [1]: [i_item_sk#22] Join type: Inner Join condition: None -(44) Project [codegen id : 9] +(44) Project [codegen id : 7] Output [3]: [rnk#11, i_product_name#21 AS best_performing#24, i_product_name#23 AS worst_performing#25] Input [5]: [rnk#11, item_sk#16, i_product_name#21, i_item_sk#22, i_product_name#23] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/extended.txt index 6b58597252..7e28f0cb5b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/extended.txt @@ -51,12 +51,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item 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 d0c3ad435b..75bbcfd60a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (9) + WholeStageCodegen (7) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -65,11 +65,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ReusedExchange [ss_item_sk,sum,count] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] + CometNativeColumnarToRow + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/explain.txt index 2114d6e892..d07ded1904 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * Project (34) @@ -19,22 +19,22 @@ TakeOrderedAndProject (39) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.web_sales (1) : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) + : : : : +- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.customer (4) : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometProject (12) : : : +- CometFilter (11) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) : : +- ReusedExchange (17) : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometProject (22) : +- CometFilter (21) : +- CometNativeScan parquet spark_catalog.default.item (20) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.item (27) @@ -48,10 +48,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 1] Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) @@ -66,20 +66,20 @@ ReadSchema: struct Input [2]: [c_customer_sk#7, c_current_addr_sk#8] Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [2]: [c_customer_sk#7, c_current_addr_sk#8] (7) BroadcastExchange Input [2]: [c_customer_sk#7, c_current_addr_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_bill_customer_sk#3] Right keys [1]: [c_customer_sk#7] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 1] Output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] @@ -98,33 +98,33 @@ Condition : isnotnull(ca_address_sk#9) Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#11, 10, true, false, true) AS ca_zip#12] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] (14) BroadcastExchange Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#8] Right keys [1]: [ca_address_sk#9] Join type: Inner Join condition: None -(16) Project [codegen id : 6] +(16) Project [codegen id : 1] Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] (17) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#13] -(18) BroadcastHashJoin [codegen id : 6] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#5] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(19) Project [codegen id : 1] Output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] @@ -143,20 +143,20 @@ Condition : isnotnull(i_item_sk#14) Input [2]: [i_item_sk#14, i_item_id#15] Arguments: [i_item_sk#14, i_item_id#16], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#16] -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [i_item_sk#14, i_item_id#16] (24) BroadcastExchange Input [2]: [i_item_sk#14, i_item_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#2] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(26) Project [codegen id : 1] Output [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16] Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#14, i_item_id#16] @@ -175,28 +175,28 @@ Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#17, i_item_id#18] Arguments: [i_item_id#19], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#18, 16, true, false, true) AS i_item_id#19] -(30) CometColumnarToRow [codegen id : 5] +(30) CometNativeColumnarToRow Input [1]: [i_item_id#19] (31) BroadcastExchange Input [1]: [i_item_id#19] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 6] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [i_item_id#16] Right keys [1]: [i_item_id#19] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 6] +(33) Filter [codegen id : 1] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(34) Project [codegen id : 6] +(34) Project [codegen id : 1] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] -(35) HashAggregate [codegen id : 6] +(35) HashAggregate [codegen id : 1] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] @@ -207,10 +207,10 @@ Results [3]: [ca_zip#12, ca_city#10, sum#21] Input [3]: [ca_zip#12, ca_city#10, sum#21] Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 7] +(37) CometNativeColumnarToRow Input [3]: [ca_zip#12, ca_city#10, sum#21] -(38) HashAggregate [codegen id : 7] +(38) HashAggregate [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#21] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] @@ -225,7 +225,7 @@ Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometNativeScan parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2) Input [3]: [d_date_sk#13, d_year#24, d_qoy#25] Arguments: [d_date_sk#13], [d_date_sk#13] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt index c58a64e326..5b686ba177 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -19,31 +19,31 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/simplified.txt index 81ecde84b2..7a0bb62bd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) + WholeStageCodegen (1) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] @@ -23,42 +23,32 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + CometNativeColumnarToRow + CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_city,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/explain.txt index f6ab9a8d53..e69785567f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/explain.txt @@ -34,7 +34,7 @@ TakeOrderedAndProject (39) : +- CometFilter (21) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) @@ -160,7 +160,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] -(26) CometColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 1] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -178,28 +178,28 @@ Condition : i_item_sk#19 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#19, i_item_id#20] Arguments: [i_item_id#21], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [i_item_id#21] (31) BroadcastExchange Input [1]: [i_item_id#21] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] -(32) BroadcastHashJoin [codegen id : 2] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [i_item_id#18] Right keys [1]: [i_item_id#21] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 2] +(33) Filter [codegen id : 1] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(34) Project [codegen id : 2] +(34) Project [codegen id : 1] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -(35) HashAggregate [codegen id : 2] +(35) HashAggregate [codegen id : 1] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] @@ -210,10 +210,10 @@ Results [3]: [ca_zip#12, ca_city#10, sum#23] Input [3]: [ca_zip#12, ca_city#10, sum#23] Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(37) CometColumnarToRow [codegen id : 3] +(37) CometColumnarToRow [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#23] -(38) HashAggregate [codegen id : 3] +(38) HashAggregate [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#23] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] @@ -228,7 +228,7 @@ Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -249,7 +249,7 @@ Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2) Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt index 4a57091056..23662b8aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/simplified.txt index d78fc716ab..2d2ba0c558 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] @@ -23,12 +23,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] @@ -46,9 +44,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [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] + CometNativeColumnarToRow + CometProject [i_item_id] [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt index f6ab9a8d53..e69785567f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt @@ -34,7 +34,7 @@ TakeOrderedAndProject (39) : +- CometFilter (21) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) @@ -160,7 +160,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] -(26) CometColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 1] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -178,28 +178,28 @@ Condition : i_item_sk#19 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#19, i_item_id#20] Arguments: [i_item_id#21], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [i_item_id#21] (31) BroadcastExchange Input [1]: [i_item_id#21] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] -(32) BroadcastHashJoin [codegen id : 2] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [i_item_id#18] Right keys [1]: [i_item_id#21] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 2] +(33) Filter [codegen id : 1] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(34) Project [codegen id : 2] +(34) Project [codegen id : 1] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -(35) HashAggregate [codegen id : 2] +(35) HashAggregate [codegen id : 1] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] @@ -210,10 +210,10 @@ Results [3]: [ca_zip#12, ca_city#10, sum#23] Input [3]: [ca_zip#12, ca_city#10, sum#23] Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(37) CometColumnarToRow [codegen id : 3] +(37) CometColumnarToRow [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#23] -(38) HashAggregate [codegen id : 3] +(38) HashAggregate [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#23] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] @@ -228,7 +228,7 @@ Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -249,7 +249,7 @@ Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2) Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/extended.txt index 4a57091056..23662b8aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt index d78fc716ab..2d2ba0c558 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] @@ -23,12 +23,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] @@ -46,9 +44,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [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] + CometNativeColumnarToRow + CometProject [i_item_id] [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/explain.txt index 994cd12469..40ba9461cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (41) :- * Project (37) : +- * BroadcastHashJoin Inner BuildRight (36) : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) + : : +- CometNativeColumnarToRow (29) : : +- CometColumnarExchange (28) : : +- * HashAggregate (27) : : +- * Project (26) @@ -21,21 +21,21 @@ TakeOrderedAndProject (41) : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : +- ReusedExchange (4) : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) + : : : : +- CometNativeColumnarToRow (10) : : : : +- CometProject (9) : : : : +- CometFilter (8) : : : : +- CometNativeScan parquet spark_catalog.default.store (7) : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) + : : : +- CometNativeColumnarToRow (17) : : : +- CometProject (16) : : : +- CometFilter (15) : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometFilter (22) : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) + : +- CometNativeColumnarToRow (34) : +- CometProject (33) : +- CometFilter (32) : +- CometNativeScan parquet spark_catalog.default.customer (31) @@ -50,23 +50,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#10] -(5) BroadcastHashJoin [codegen id : 5] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(6) Project [codegen id : 5] +(6) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] @@ -85,20 +85,20 @@ Condition : (s_city#12 IN (Fairview,Midway) AND isnotnull(s_store_sk#11)) Input [2]: [s_store_sk#11, s_city#12] Arguments: [s_store_sk#11], [s_store_sk#11] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#11] (11) BroadcastExchange Input [1]: [s_store_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11] @@ -117,20 +117,20 @@ Condition : (((hd_dep_count#14 = 4) OR (hd_vehicle_count#15 = 3)) AND isnotnull( Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] Arguments: [hd_demo_sk#13], [hd_demo_sk#13] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#13] (18) BroadcastExchange Input [1]: [hd_demo_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#13] @@ -145,24 +145,24 @@ ReadSchema: struct Input [2]: [ca_address_sk#16, ca_city#17] Condition : (isnotnull(ca_address_sk#16) AND isnotnull(ca_city#17)) -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [ca_address_sk#16, ca_city#17] (24) BroadcastExchange Input [2]: [ca_address_sk#16, ca_city#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(26) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#16, ca_city#17] -(27) HashAggregate [codegen id : 5] +(27) HashAggregate [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] @@ -173,10 +173,10 @@ Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, su Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometColumnarToRow [codegen id : 8] +(29) CometNativeColumnarToRow Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -(30) HashAggregate [codegen id : 8] +(30) HashAggregate [codegen id : 2] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] @@ -198,33 +198,33 @@ Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] Arguments: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32], [c_customer_sk#27, c_current_addr_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#32] -(34) CometColumnarToRow [codegen id : 6] +(34) CometNativeColumnarToRow Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] (35) BroadcastExchange Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#27] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 2] Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32] Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] (38) ReusedExchange [Reuses operator id: 24] Output [2]: [ca_address_sk#33, ca_city#34] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_current_addr_sk#28] Right keys [1]: [ca_address_sk#33] Join type: Inner Join condition: NOT (ca_city#34 = bought_city#24) -(40) Project [codegen id : 8] +(40) Project [codegen id : 2] Output [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] @@ -236,7 +236,7 @@ Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometNativeScan parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((d_dow#36 IN (6,0) AND d_year#35 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#35, d_dow#36] Arguments: [d_date_sk#10], [d_date_sk#10] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt index c122bf3803..cebcd85144 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -20,36 +20,36 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/simplified.txt index b4b78689b1..2b3e7984ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -24,44 +24,34 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometNativeColumnarToRow + CometFilter [ca_address_sk,ca_city] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/explain.txt index ca7ae9df0c..044048fc80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/explain.txt @@ -225,7 +225,7 @@ Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/simplified.txt index e646cb620a..59123d0f6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt index ca7ae9df0c..044048fc80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt @@ -225,7 +225,7 @@ Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e646cb620a..59123d0f6f 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/explain.txt index 3f98e1bd21..0cf411ed0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.store_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.store (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.store_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#6] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] (17) BroadcastExchange Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#7] Right keys [1]: [s_store_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] @@ -151,10 +151,10 @@ Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_yea Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] (27) Window Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] (32) ReusedExchange [Reuses operator id: 21] Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_ Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] (38) Window Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] @@ -228,13 +228,13 @@ Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year# Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] @@ -245,14 +245,14 @@ Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] (46) Window Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] @@ -260,13 +260,13 @@ Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year# Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt index 092083e465..ab44bbef5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/simplified.txt index 017805e5b2..139a34baee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk,s_store_name,s_company_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/explain.txt index 1c039203ce..dce7dc0840 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/simplified.txt index 19124a9e20..56fdda3162 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt index 1c039203ce..dce7dc0840 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 19124a9e20..56fdda3162 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/explain.txt index 494c1d4338..a582c713ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * HashAggregate (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometColumnarExchange (28) +- * HashAggregate (27) +- * Project (26) @@ -15,16 +15,16 @@ : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) + : : : +- CometNativeColumnarToRow (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.store (4) : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) + : : +- CometNativeColumnarToRow (13) : : +- CometProject (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (10) : +- BroadcastExchange (21) - : +- * CometColumnarToRow (20) + : +- CometNativeColumnarToRow (20) : +- CometProject (19) : +- CometFilter (18) : +- CometNativeScan parquet spark_catalog.default.customer_address (17) @@ -39,10 +39,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) @@ -57,20 +57,20 @@ ReadSchema: struct Input [1]: [s_store_sk#9] Condition : isnotnull(s_store_sk#9) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [s_store_sk#9] (7) BroadcastExchange Input [1]: [s_store_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 5] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#9] Join type: Inner Join condition: None -(9) Project [codegen id : 5] +(9) Project [codegen id : 1] Output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] @@ -89,20 +89,20 @@ Condition : (isnotnull(cd_demo_sk#10) AND ((((staticinvoke(class org.apache.spar Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) AS cd_marital_status#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) AS cd_education_status#14] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] (14) BroadcastExchange Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 5] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) -(16) Project [codegen id : 5] +(16) Project [codegen id : 1] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] @@ -121,37 +121,37 @@ Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#18] -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [2]: [ca_address_sk#15, ca_state#18] (21) BroadcastExchange Input [2]: [ca_address_sk#15, ca_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 5] +(22) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#15] Join type: Inner Join condition: ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))) -(23) Project [codegen id : 5] +(23) Project [codegen id : 1] Output [2]: [ss_quantity#4, ss_sold_date_sk#7] Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] (24) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#19] -(25) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(26) Project [codegen id : 1] Output [1]: [ss_quantity#4] Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] -(27) HashAggregate [codegen id : 5] +(27) HashAggregate [codegen id : 1] Input [1]: [ss_quantity#4] Keys: [] Functions [1]: [partial_sum(ss_quantity#4)] @@ -162,10 +162,10 @@ Results [1]: [sum#21] Input [1]: [sum#21] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometColumnarToRow [codegen id : 6] +(29) CometNativeColumnarToRow Input [1]: [sum#21] -(30) HashAggregate [codegen id : 6] +(30) HashAggregate [codegen id : 2] Input [1]: [sum#21] Keys: [] Functions [1]: [sum(ss_quantity#4)] @@ -176,7 +176,7 @@ Results [1]: [sum(ss_quantity#4)#22 AS sum(ss_quantity)#23] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#24) AND (d_year#24 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#19, d_year#24] Arguments: [d_date_sk#19], [d_date_sk#19] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt index 1ef8426231..5e6a26863a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,26 +15,26 @@ HashAggregate : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/simplified.txt index 0cd7e900b0..b258895829 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (6) +WholeStageCodegen (2) HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_quantity] [sum,sum] Project [ss_quantity] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -19,34 +19,26 @@ WholeStageCodegen (6) Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/explain.txt index 71924d1e6d..3afd893173 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/explain.txt @@ -171,7 +171,7 @@ Input [1]: [sum(ss_quantity)#22] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -192,7 +192,7 @@ Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#19, d_year#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt index 57fbc2d035..347702b724 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/simplified.txt index e29e101c7e..52cdd88b76 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/simplified.txt @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt index 71924d1e6d..3afd893173 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt @@ -171,7 +171,7 @@ Input [1]: [sum(ss_quantity)#22] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -192,7 +192,7 @@ Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#19, d_year#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/extended.txt index 57fbc2d035..347702b724 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e29e101c7e..52cdd88b76 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 @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/explain.txt index 0a43ebc390..90cb3f712c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (84) +CometNativeColumnarToRow (84) +- CometTakeOrderedAndProject (83) +- CometHashAggregate (82) +- CometColumnarExchange (81) @@ -10,11 +10,11 @@ : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometSort (20) : +- CometColumnarExchange (19) : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometColumnarExchange (16) : +- * HashAggregate (15) : +- * Project (14) @@ -26,7 +26,7 @@ : : : +- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) + : : +- CometNativeColumnarToRow (9) : : +- CometProject (8) : : +- CometFilter (7) : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) @@ -36,11 +36,11 @@ : +- Window (50) : +- * Sort (49) : +- Window (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometSort (46) : +- CometColumnarExchange (45) : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -52,7 +52,7 @@ : : : +- * Filter (29) : : : +- * ColumnarToRow (28) : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) + : : +- CometNativeColumnarToRow (35) : : +- CometProject (34) : : +- CometFilter (33) : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) @@ -62,11 +62,11 @@ +- Window (76) +- * Sort (75) +- Window (74) - +- * CometColumnarToRow (73) + +- CometNativeColumnarToRow (73) +- CometSort (72) +- CometColumnarExchange (71) +- * HashAggregate (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometColumnarExchange (68) +- * HashAggregate (67) +- * Project (66) @@ -78,7 +78,7 @@ : : +- * Filter (55) : : +- * ColumnarToRow (54) : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) + : +- CometNativeColumnarToRow (61) : +- CometProject (60) : +- CometFilter (59) : +- CometNativeScan parquet spark_catalog.default.store_returns (58) @@ -123,33 +123,33 @@ Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AN Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometColumnarToRow +(9) CometNativeColumnarToRow Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [2]: [ws_order_number#2, ws_item_sk#1] Right keys [2]: [wr_order_number#9, wr_item_sk#8] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] (12) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#13] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#6] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 2] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -160,10 +160,10 @@ Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty# Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -178,14 +178,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 5] +(21) CometNativeColumnarToRow Input [3]: [item#30, return_ratio#31, currency_ratio#32] (22) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 4] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 @@ -193,11 +193,11 @@ Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 5] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(26) Project [codegen id : 7] +(26) Project [codegen id : 5] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -209,14 +209,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -(29) Filter [codegen id : 8] +(29) Filter [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(30) Project [codegen id : 8] +(30) Project [codegen id : 6] Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] @@ -239,33 +239,33 @@ Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.0 Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(35) CometColumnarToRow +(35) CometNativeColumnarToRow Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 7] Left keys [2]: [cs_order_number#37, cs_item_sk#36] Right keys [2]: [cr_order_number#43, cr_item_sk#42] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 7] Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] (38) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#47] -(39) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#47] Join type: Inner Join condition: None -(40) Project [codegen id : 10] +(40) Project [codegen id : 7] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] -(41) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -276,10 +276,10 @@ Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometColumnarToRow [codegen id : 11] +(43) CometNativeColumnarToRow Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 8] Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -294,14 +294,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 12] +(47) CometNativeColumnarToRow Input [3]: [item#64, return_ratio#65, currency_ratio#66] (48) Window Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] -(49) Sort [codegen id : 13] +(49) Sort [codegen id : 9] Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 @@ -309,11 +309,11 @@ Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] -(51) Filter [codegen id : 14] +(51) Filter [codegen id : 10] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) -(52) Project [codegen id : 14] +(52) Project [codegen id : 10] Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] @@ -325,14 +325,14 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 15] +(54) ColumnarToRow [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -(55) Filter [codegen id : 15] +(55) Filter [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) -(56) Project [codegen id : 15] +(56) Project [codegen id : 11] Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] @@ -355,33 +355,33 @@ Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AN Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(61) CometColumnarToRow +(61) CometNativeColumnarToRow Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(62) BroadcastHashJoin [codegen id : 17] +(62) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(63) Project [codegen id : 12] Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] (64) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#81] -(65) BroadcastHashJoin [codegen id : 17] +(65) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ss_sold_date_sk#75] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(66) Project [codegen id : 12] Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] -(67) HashAggregate [codegen id : 17] +(67) HashAggregate [codegen id : 12] Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Keys [1]: [ss_item_sk#70] Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -392,10 +392,10 @@ Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(69) CometColumnarToRow [codegen id : 18] +(69) CometNativeColumnarToRow Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -(70) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 13] Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Keys [1]: [ss_item_sk#70] Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -410,14 +410,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [3]: [item#98, return_ratio#99, currency_ratio#100] (74) Window Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] -(75) Sort [codegen id : 20] +(75) Sort [codegen id : 14] Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 @@ -425,17 +425,17 @@ Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] -(77) Filter [codegen id : 21] +(77) Filter [codegen id : 15] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) -(78) Project [codegen id : 21] +(78) Project [codegen id : 15] Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] (79) Union -(80) HashAggregate [codegen id : 22] +(80) HashAggregate [codegen id : 16] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] @@ -455,14 +455,14 @@ Functions: [] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) CometColumnarToRow [codegen id : 23] +(84) CometNativeColumnarToRow Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometNativeScan parquet spark_catalog.default.date_dim (85) @@ -483,7 +483,7 @@ Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 Input [3]: [d_date_sk#13, d_year#104, d_moy#105] Arguments: [d_date_sk#13], [d_date_sk#13] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt index 6c4ded1515..0090739d55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -9,11 +9,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -27,16 +27,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -45,11 +45,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -62,12 +62,12 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -76,11 +76,11 @@ CometColumnarToRow +- Window +- Sort +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -93,12 +93,12 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/simplified.txt index 047269434d..018aec1d47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/simplified.txt @@ -1,140 +1,130 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + CometNativeColumnarToRow + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #6 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [cs_item_sk] #7 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (15) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #9 + WholeStageCodegen (13) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (12) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (11) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/explain.txt index 338f5f4201..0ecba1505f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/simplified.txt index ca80833ee5..3353e1c331 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/simplified.txt @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt index 338f5f4201..0ecba1505f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ca80833ee5..3353e1c331 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 @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/explain.txt index a0a95bd28b..c0d72cc899 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == TakeOrderedAndProject (81) +- * HashAggregate (80) - +- * CometColumnarToRow (79) + +- CometNativeColumnarToRow (79) +- CometColumnarExchange (78) +- * HashAggregate (77) +- * Expand (76) +- Union (75) :- * HashAggregate (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometColumnarExchange (21) : +- * HashAggregate (20) : +- * Project (19) @@ -25,12 +25,12 @@ TakeOrderedAndProject (81) : : : +- Scan parquet spark_catalog.default.store_returns (5) : : +- ReusedExchange (10) : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometProject (15) : +- CometFilter (14) : +- CometNativeScan parquet spark_catalog.default.store (13) :- * HashAggregate (46) - : +- * CometColumnarToRow (45) + : +- CometNativeColumnarToRow (45) : +- CometColumnarExchange (44) : +- * HashAggregate (43) : +- * Project (42) @@ -48,12 +48,12 @@ TakeOrderedAndProject (81) : : : +- Scan parquet spark_catalog.default.catalog_returns (28) : : +- ReusedExchange (33) : +- BroadcastExchange (40) - : +- * CometColumnarToRow (39) + : +- CometNativeColumnarToRow (39) : +- CometProject (38) : +- CometFilter (37) : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) +- * HashAggregate (74) - +- * CometColumnarToRow (73) + +- CometNativeColumnarToRow (73) +- CometColumnarExchange (72) +- * HashAggregate (71) +- * Project (70) @@ -70,13 +70,13 @@ TakeOrderedAndProject (81) : : :- BroadcastExchange (53) : : : +- * ColumnarToRow (52) : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : +- * CometColumnarToRow (57) + : : +- CometNativeColumnarToRow (57) : : +- CometProject (56) : : +- CometFilter (55) : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) : +- ReusedExchange (61) +- BroadcastExchange (68) - +- * CometColumnarToRow (67) + +- CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometNativeScan parquet spark_catalog.default.web_site (64) @@ -125,13 +125,13 @@ Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_s (10) ReusedExchange [Reuses operator id: 86] Output [1]: [d_date_sk#22] -(11) BroadcastHashJoin [codegen id : 5] +(11) BroadcastHashJoin [codegen id : 3] Left keys [1]: [date_sk#7] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(12) Project [codegen id : 5] +(12) Project [codegen id : 3] Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] @@ -150,24 +150,24 @@ Condition : isnotnull(s_store_sk#23) Input [2]: [s_store_sk#23, s_store_id#24] Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#23, s_store_id#25] (17) BroadcastExchange Input [2]: [s_store_sk#23, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 5] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [store_sk#6] Right keys [1]: [s_store_sk#23] Join type: Inner Join condition: None -(19) Project [codegen id : 5] +(19) Project [codegen id : 3] Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] -(20) HashAggregate [codegen id : 5] +(20) HashAggregate [codegen id : 3] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] @@ -178,10 +178,10 @@ Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(23) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 4] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -196,14 +196,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] +(25) ColumnarToRow [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -(26) Filter [codegen id : 7] +(26) Filter [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Condition : isnotnull(cs_catalog_page_sk#43) -(27) Project [codegen id : 7] +(27) Project [codegen id : 5] Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] @@ -215,14 +215,14 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 8] +(29) ColumnarToRow [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -(30) Filter [codegen id : 8] +(30) Filter [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(31) Project [codegen id : 8] +(31) Project [codegen id : 6] Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] @@ -231,13 +231,13 @@ Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_retur (33) ReusedExchange [Reuses operator id: 86] Output [1]: [d_date_sk#63] -(34) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(35) Project [codegen id : 11] +(35) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -256,24 +256,24 @@ Condition : isnotnull(cp_catalog_page_sk#64) Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#65, 16, true, false, true) AS cp_catalog_page_id#66] -(39) CometColumnarToRow [codegen id : 10] +(39) CometNativeColumnarToRow Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] (40) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(41) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(42) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] @@ -284,10 +284,10 @@ Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(45) CometColumnarToRow [codegen id : 12] +(45) CometNativeColumnarToRow Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -(46) HashAggregate [codegen id : 12] +(46) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -302,14 +302,14 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 13] +(48) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(49) Filter [codegen id : 13] +(49) Filter [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Condition : isnotnull(ws_web_site_sk#84) -(50) Project [codegen id : 13] +(50) Project [codegen id : 9] Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] @@ -320,7 +320,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 14] +(52) ColumnarToRow [codegen id : 10] Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (53) BroadcastExchange @@ -342,16 +342,16 @@ Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND i Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(57) CometColumnarToRow +(57) CometNativeColumnarToRow Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(58) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 11] Left keys [2]: [wr_item_sk#94, wr_order_number#95] Right keys [2]: [ws_item_sk#99, ws_order_number#101] Join type: Inner Join condition: None -(59) Project [codegen id : 15] +(59) Project [codegen id : 11] Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] @@ -360,13 +360,13 @@ Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, (61) ReusedExchange [Reuses operator id: 86] Output [1]: [d_date_sk#109] -(62) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 12] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(63) Project [codegen id : 18] +(63) Project [codegen id : 12] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -385,24 +385,24 @@ Condition : isnotnull(web_site_sk#110) Input [2]: [web_site_sk#110, web_site_id#111] Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#111, 16, true, false, true) AS web_site_id#112] -(67) CometColumnarToRow [codegen id : 17] +(67) CometNativeColumnarToRow Input [2]: [web_site_sk#110, web_site_id#112] (68) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(69) BroadcastHashJoin [codegen id : 18] +(69) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(70) Project [codegen id : 18] +(70) Project [codegen id : 12] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] -(71) HashAggregate [codegen id : 18] +(71) HashAggregate [codegen id : 12] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Keys [1]: [web_site_id#112] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] @@ -413,10 +413,10 @@ Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -(74) HashAggregate [codegen id : 19] +(74) HashAggregate [codegen id : 13] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Keys [1]: [web_site_id#112] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] @@ -425,11 +425,11 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales# (75) Union -(76) Expand [codegen id : 20] +(76) Expand [codegen id : 14] Input [5]: [sales#38, returns#39, profit#40, channel#41, id#42] Arguments: [[sales#38, returns#39, profit#40, channel#41, id#42, 0], [sales#38, returns#39, profit#40, channel#41, null, 1], [sales#38, returns#39, profit#40, null, null, 3]], [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] -(77) HashAggregate [codegen id : 20] +(77) HashAggregate [codegen id : 14] Input [6]: [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] Keys [3]: [channel#130, id#131, spark_grouping_id#132] Functions [3]: [partial_sum(sales#38), partial_sum(returns#39), partial_sum(profit#40)] @@ -440,10 +440,10 @@ Results [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 21] +(79) CometNativeColumnarToRow Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -(80) HashAggregate [codegen id : 21] +(80) HashAggregate [codegen id : 15] Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Keys [3]: [channel#130, id#131, spark_grouping_id#132] Functions [3]: [sum(sales#38), sum(returns#39), sum(profit#40)] @@ -458,7 +458,7 @@ Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel# Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (86) -+- * CometColumnarToRow (85) ++- CometNativeColumnarToRow (85) +- CometProject (84) +- CometFilter (83) +- CometNativeScan parquet spark_catalog.default.date_dim (82) @@ -479,7 +479,7 @@ Condition : (((isnotnull(d_date#151) AND (d_date#151 >= 2000-08-23)) AND (d_date Input [2]: [d_date_sk#22, d_date#151] Arguments: [d_date_sk#22], [d_date_sk#22] -(85) CometColumnarToRow [codegen id : 1] +(85) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (86) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt index 60352ef90a..15ad014521 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -30,17 +30,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -59,17 +59,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.catalog_page +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -88,17 +88,17 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/simplified.txt index 9dc4fc19c7..d84ba471ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (21) + WholeStageCodegen (15) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (20) + WholeStageCodegen (14) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_id] #2 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] BroadcastHashJoin [store_sk,s_store_sk] @@ -30,12 +30,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] WholeStageCodegen (2) Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_store_sk] @@ -47,18 +45,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cp_catalog_page_id] #5 - WholeStageCodegen (11) + WholeStageCodegen (7) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] BroadcastHashJoin [page_sk,cp_catalog_page_sk] @@ -66,14 +62,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (5) Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] Filter [cs_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) + WholeStageCodegen (6) Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] Filter [cr_catalog_page_sk] ColumnarToRow @@ -84,18 +80,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) + CometNativeColumnarToRow + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (13) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [web_site_id] #7 - WholeStageCodegen (18) + WholeStageCodegen (12) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] BroadcastHashJoin [wsr_web_site_sk,web_site_sk] @@ -103,25 +97,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union - WholeStageCodegen (13) + WholeStageCodegen (9) Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (15) + WholeStageCodegen (11) Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] InputAdapter BroadcastExchange #8 - WholeStageCodegen (14) + WholeStageCodegen (10) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] @@ -129,9 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometNativeColumnarToRow + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt index b4a1dedd61..d86780b8ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt @@ -411,7 +411,7 @@ Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -432,7 +432,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt index d128acaf06..76c166e7de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt index 6fff89a25c..aa3a803ef1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt index b4a1dedd61..d86780b8ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt @@ -411,7 +411,7 @@ Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -432,7 +432,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/extended.txt index d128acaf06..76c166e7de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6fff89a25c..aa3a803ef1 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/explain.txt index de9a1cd3ec..188ce4e5af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (30) +- * HashAggregate (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometColumnarExchange (27) +- * HashAggregate (26) +- * Project (25) @@ -12,7 +12,7 @@ TakeOrderedAndProject (30) : : +- * BroadcastHashJoin Inner BuildRight (15) : : :- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * CometColumnarToRow (3) + : : : :- CometNativeColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (7) @@ -20,12 +20,12 @@ TakeOrderedAndProject (30) : : : +- * ColumnarToRow (5) : : : +- Scan parquet spark_catalog.default.store_returns (4) : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) + : : +- CometNativeColumnarToRow (13) : : +- CometProject (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.store (10) : +- BroadcastExchange (20) - : +- * CometColumnarToRow (19) + : +- CometNativeColumnarToRow (19) : +- CometFilter (18) : +- CometNativeScan parquet spark_catalog.default.date_dim (17) +- ReusedExchange (23) @@ -43,7 +43,7 @@ ReadSchema: struct Input [1]: [d_date_sk#26] Condition : isnotnull(d_date_sk#26) -(19) CometColumnarToRow [codegen id : 3] +(19) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (20) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(21) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(22) Project [codegen id : 5] +(22) Project [codegen id : 2] Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] (23) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#27] -(24) BroadcastHashJoin [codegen id : 5] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#9] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(25) Project [codegen id : 5] +(25) Project [codegen id : 2] Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 2] Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] @@ -159,10 +159,10 @@ Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_na Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] -(29) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 3] Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] @@ -177,7 +177,7 @@ Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRS Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -198,7 +198,7 @@ Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2 Input [3]: [d_date_sk#27, d_year#48, d_moy#49] Arguments: [d_date_sk#27], [d_date_sk#27] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#27] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt index 6ece21441c..ab41730455 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- BroadcastExchange @@ -20,21 +20,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/simplified.txt index 186c041b09..fddccaf3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -14,8 +14,8 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] InputAdapter @@ -27,26 +27,20 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometNativeColumnarToRow + CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk] + CometNativeColumnarToRow + CometFilter [d_date_sk] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk] InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/explain.txt index ff37c8f5fb..0ed16904e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/explain.txt @@ -172,7 +172,7 @@ Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -193,7 +193,7 @@ Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2 Input [3]: [d_date_sk#27, d_year#28, d_moy#29] Arguments: [d_date_sk#27], [d_date_sk#27] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#27] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt index 0fbba7d1e8..958b844c9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/simplified.txt index 98a44eaf40..fd3cd6eaa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt index ff37c8f5fb..0ed16904e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt @@ -172,7 +172,7 @@ Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -193,7 +193,7 @@ Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2 Input [3]: [d_date_sk#27, d_year#28, d_moy#29] Arguments: [d_date_sk#27], [d_date_sk#27] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#27] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/extended.txt index 0fbba7d1e8..958b844c9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 98a44eaf40..fd3cd6eaa5 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/explain.txt index 9af40029c8..508b20f435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (42) +- * Filter (41) +- Window (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometSort (38) +- CometExchange (37) +- CometProject (36) @@ -11,11 +11,11 @@ TakeOrderedAndProject (42) : +- CometColumnarExchange (16) : +- * Project (15) : +- Window (14) - : +- * CometColumnarToRow (13) + : +- CometNativeColumnarToRow (13) : +- CometSort (12) : +- CometColumnarExchange (11) : +- * HashAggregate (10) - : +- * CometColumnarToRow (9) + : +- CometNativeColumnarToRow (9) : +- CometColumnarExchange (8) : +- * HashAggregate (7) : +- * Project (6) @@ -28,11 +28,11 @@ TakeOrderedAndProject (42) +- CometColumnarExchange (33) +- * Project (32) +- Window (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometSort (29) +- CometColumnarExchange (28) +- * HashAggregate (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometColumnarExchange (25) +- * HashAggregate (24) +- * Project (23) @@ -51,27 +51,27 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 47] Output [2]: [d_date_sk#5, d_date#6] -(5) BroadcastHashJoin [codegen id : 2] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 2] +(6) Project [codegen id : 1] Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -(7) HashAggregate [codegen id : 2] +(7) HashAggregate [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] @@ -82,10 +82,10 @@ Results [3]: [ws_item_sk#1, d_date#6, sum#8] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 3] +(9) CometNativeColumnarToRow Input [3]: [ws_item_sk#1, d_date#6, sum#8] -(10) HashAggregate [codegen id : 3] +(10) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] @@ -100,14 +100,14 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 4] +(13) CometNativeColumnarToRow Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] (14) Window Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 5] +(15) Project [codegen id : 3] Output [3]: [item_sk#10, d_date#6, cume_sales#12] Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] @@ -127,27 +127,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 4] Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -(20) Filter [codegen id : 7] +(20) Filter [codegen id : 4] Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) (21) ReusedExchange [Reuses operator id: 47] Output [2]: [d_date_sk#16, d_date#17] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#15] Right keys [1]: [d_date_sk#16] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 4] Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16, d_date#17] -(24) HashAggregate [codegen id : 7] +(24) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] Keys [2]: [ss_item_sk#13, d_date#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] @@ -158,10 +158,10 @@ Results [3]: [ss_item_sk#13, d_date#17, sum#19] Input [3]: [ss_item_sk#13, d_date#17, sum#19] Arguments: hashpartitioning(ss_item_sk#13, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometColumnarToRow [codegen id : 8] +(26) CometNativeColumnarToRow Input [3]: [ss_item_sk#13, d_date#17, sum#19] -(27) HashAggregate [codegen id : 8] +(27) HashAggregate [codegen id : 5] Input [3]: [ss_item_sk#13, d_date#17, sum#19] Keys [2]: [ss_item_sk#13, d_date#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] @@ -176,14 +176,14 @@ Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumna Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] Arguments: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(30) CometColumnarToRow [codegen id : 9] +(30) CometNativeColumnarToRow Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] (31) Window Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#17 ASC NULLS FIRST] -(32) Project [codegen id : 10] +(32) Project [codegen id : 6] Output [3]: [item_sk#21, d_date#17, cume_sales#23] Input [5]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13, cume_sales#23] @@ -212,14 +212,14 @@ Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] -(39) CometColumnarToRow [codegen id : 11] +(39) CometNativeColumnarToRow Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] (40) Window Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(41) Filter [codegen id : 12] +(41) Filter [codegen id : 7] Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) @@ -231,7 +231,7 @@ Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (47) -+- * CometColumnarToRow (46) ++- CometNativeColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometNativeScan parquet spark_catalog.default.date_dim (43) @@ -252,7 +252,7 @@ Condition : (((isnotnull(d_month_seq#30) AND (d_month_seq#30 >= 1200)) AND (d_mo Input [3]: [d_date_sk#5, d_date#6, d_month_seq#30] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(46) CometColumnarToRow [codegen id : 1] +(46) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt index 9dc007f5e9..db4e38a1bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -10,11 +10,11 @@ TakeOrderedAndProject : +- CometColumnarExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -24,12 +24,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,11 +37,11 @@ TakeOrderedAndProject +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -51,7 +51,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/simplified.txt index 51d3f11ae6..1029c768d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/simplified.txt @@ -1,73 +1,65 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (12) + WholeStageCodegen (7) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #6 - WholeStageCodegen (10) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #8 - WholeStageCodegen (7) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + CometNativeColumnarToRow + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (3) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk,d_date] #4 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #6 + WholeStageCodegen (6) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #7 + WholeStageCodegen (5) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk,d_date] #8 + WholeStageCodegen (4) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/explain.txt index 683cc25d24..716f297aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/explain.txt @@ -224,7 +224,7 @@ Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -245,7 +245,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt index 66c5717cc7..8a2badfaff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/simplified.txt index a5af2e114b..ba02a3e822 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/simplified.txt @@ -30,12 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt index 683cc25d24..716f297aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt @@ -224,7 +224,7 @@ Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -245,7 +245,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt index 66c5717cc7..8a2badfaff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a5af2e114b..ba02a3e822 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 @@ -30,12 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/explain.txt index 4af1b4f562..3a3764351c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/simplified.txt index cee223d027..f5de3784b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_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 [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] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [d_year,brand_id,brand,ext_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 [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] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/explain.txt index f8c8dd3bf2..bf21baa1b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (30) +- * Project (29) +- * Filter (28) +- Window (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -16,7 +16,7 @@ TakeOrderedAndProject (30) : +- * BroadcastHashJoin Inner BuildRight (12) : :- * Project (10) : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject (30) : : +- Scan parquet spark_catalog.default.store_sales (5) : +- ReusedExchange (11) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -46,7 +46,7 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [2]: [i_item_sk#1, i_manufact_id#5] (5) Scan parquet spark_catalog.default.store_sales @@ -68,26 +68,26 @@ Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] (11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#15, d_qoy#16] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#13] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16] Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#16] @@ -102,24 +102,24 @@ ReadSchema: struct Input [1]: [s_store_sk#17] Condition : isnotnull(s_store_sk#17) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [s_store_sk#17] (17) BroadcastExchange Input [1]: [s_store_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#11] Right keys [1]: [s_store_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16, s_store_sk#17] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] Keys [2]: [i_manufact_id#5, d_qoy#16] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] @@ -130,10 +130,10 @@ Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Keys [2]: [i_manufact_id#5, d_qoy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] @@ -148,18 +148,18 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColum Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] (27) Window Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 7] +(29) Project [codegen id : 4] Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] @@ -171,7 +171,7 @@ Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -192,7 +192,7 @@ Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_qoy#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt index 0b6c2edaa0..5c7a26057d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject +- Project +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -25,17 +25,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/simplified.txt index 63c3e1a17a..d427ecd731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/simplified.txt @@ -1,53 +1,47 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id,d_qoy] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometNativeColumnarToRow + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + WholeStageCodegen (3) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_manufact_id,d_qoy] #2 + WholeStageCodegen (2) + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 + CometNativeColumnarToRow + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + ReusedExchange [d_date_sk,d_qoy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/explain.txt index 2fc38bf838..74b64a3eba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/simplified.txt index 0c123eb728..e0771ad655 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt index 2fc38bf838..74b64a3eba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0c123eb728..e0771ad655 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/explain.txt index bf3e53cb21..7c48f23ff6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == TakeOrderedAndProject (61) +- * HashAggregate (60) - +- * CometColumnarToRow (59) + +- CometNativeColumnarToRow (59) +- CometColumnarExchange (58) +- * HashAggregate (57) +- * HashAggregate (56) - +- * CometColumnarToRow (55) + +- CometNativeColumnarToRow (55) +- CometColumnarExchange (54) +- * HashAggregate (53) +- * Project (52) @@ -16,7 +16,7 @@ TakeOrderedAndProject (61) : : +- * BroadcastHashJoin Inner BuildRight (41) : : :- * Project (35) : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * CometColumnarToRow (29) + : : : :- CometNativeColumnarToRow (29) : : : : +- CometHashAggregate (28) : : : : +- CometColumnarExchange (27) : : : : +- * HashAggregate (26) @@ -36,13 +36,13 @@ TakeOrderedAndProject (61) : : : : : : : +- * ColumnarToRow (6) : : : : : : : +- Scan parquet spark_catalog.default.web_sales (5) : : : : : : +- BroadcastExchange (14) - : : : : : : +- * CometColumnarToRow (13) + : : : : : : +- CometNativeColumnarToRow (13) : : : : : : +- CometProject (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- ReusedExchange (17) : : : : +- BroadcastExchange (23) - : : : : +- * CometColumnarToRow (22) + : : : : +- CometNativeColumnarToRow (22) : : : : +- CometFilter (21) : : : : +- CometNativeScan parquet spark_catalog.default.customer (20) : : : +- BroadcastExchange (33) @@ -50,12 +50,12 @@ TakeOrderedAndProject (61) : : : +- * ColumnarToRow (31) : : : +- Scan parquet spark_catalog.default.store_sales (30) : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) + : : +- CometNativeColumnarToRow (39) : : +- CometProject (38) : : +- CometFilter (37) : : +- CometNativeScan parquet spark_catalog.default.customer_address (36) : +- BroadcastExchange (47) - : +- * CometColumnarToRow (46) + : +- CometNativeColumnarToRow (46) : +- CometProject (45) : +- CometFilter (44) : +- CometNativeScan parquet spark_catalog.default.store (43) @@ -117,33 +117,33 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [3]: [i_item_sk#14, i_class#15, i_category#16] Arguments: [i_item_sk#14], [i_item_sk#14] -(13) CometColumnarToRow [codegen id : 3] +(13) CometNativeColumnarToRow Input [1]: [i_item_sk#14] (14) BroadcastExchange Input [1]: [i_item_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(15) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(16) Project [codegen id : 6] +(16) Project [codegen id : 3] Output [2]: [sold_date_sk#5, customer_sk#6] Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] (17) ReusedExchange [Reuses operator id: 66] Output [1]: [d_date_sk#17] -(18) BroadcastHashJoin [codegen id : 6] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sold_date_sk#5] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(19) Project [codegen id : 3] Output [1]: [customer_sk#6] Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] @@ -158,24 +158,24 @@ ReadSchema: struct Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [2]: [c_customer_sk#18, c_current_addr_sk#19] (23) BroadcastExchange Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 3] Left keys [1]: [customer_sk#6] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 3] Output [2]: [c_customer_sk#18, c_current_addr_sk#19] Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 3] Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] Functions: [] @@ -191,7 +191,7 @@ Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] Functions: [] -(29) CometColumnarToRow [codegen id : 11] +(29) CometNativeColumnarToRow Input [2]: [c_customer_sk#18, c_current_addr_sk#19] (30) Scan parquet spark_catalog.default.store_sales @@ -202,10 +202,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] +(31) ColumnarToRow [codegen id : 4] Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -(32) Filter [codegen id : 7] +(32) Filter [codegen id : 4] Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) @@ -213,13 +213,13 @@ Condition : isnotnull(ss_customer_sk#20) Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#18] Right keys [1]: [ss_customer_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 11] +(35) Project [codegen id : 5] Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] @@ -238,20 +238,20 @@ Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnot Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] Arguments: [ca_address_sk#24, ca_county#25, ca_state#27], [ca_address_sk#24, ca_county#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#26, 2, true, false, true) AS ca_state#27] -(39) CometColumnarToRow [codegen id : 8] +(39) CometNativeColumnarToRow Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] (40) BroadcastExchange Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#19] Right keys [1]: [ca_address_sk#24] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(42) Project [codegen id : 5] Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27] Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#27] @@ -270,37 +270,37 @@ Condition : (isnotnull(s_county#28) AND isnotnull(staticinvoke(class org.apache. Input [2]: [s_county#28, s_state#29] Arguments: [s_county#28, s_state#30], [s_county#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#29, 2, true, false, true) AS s_state#30] -(46) CometColumnarToRow [codegen id : 9] +(46) CometNativeColumnarToRow Input [2]: [s_county#28, s_state#30] (47) BroadcastExchange Input [2]: [s_county#28, s_state#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 5] Left keys [2]: [ca_county#25, ca_state#27] Right keys [2]: [s_county#28, s_state#30] Join type: Inner Join condition: None -(49) Project [codegen id : 11] +(49) Project [codegen id : 5] Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27, s_county#28, s_state#30] (50) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#31] -(51) BroadcastHashJoin [codegen id : 11] +(51) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(52) Project [codegen id : 11] +(52) Project [codegen id : 5] Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#31] -(53) HashAggregate [codegen id : 11] +(53) HashAggregate [codegen id : 5] Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] Keys [1]: [c_customer_sk#18] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] @@ -311,17 +311,17 @@ Results [2]: [c_customer_sk#18, sum#33] Input [2]: [c_customer_sk#18, sum#33] Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(55) CometColumnarToRow [codegen id : 12] +(55) CometNativeColumnarToRow Input [2]: [c_customer_sk#18, sum#33] -(56) HashAggregate [codegen id : 12] +(56) HashAggregate [codegen id : 6] Input [2]: [c_customer_sk#18, sum#33] Keys [1]: [c_customer_sk#18] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#34] Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#34,17,2) / 50) as int) AS segment#35] -(57) HashAggregate [codegen id : 12] +(57) HashAggregate [codegen id : 6] Input [1]: [segment#35] Keys [1]: [segment#35] Functions [1]: [partial_count(1)] @@ -332,10 +332,10 @@ Results [2]: [segment#35, count#37] Input [2]: [segment#35, count#37] Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(59) CometColumnarToRow [codegen id : 13] +(59) CometNativeColumnarToRow Input [2]: [segment#35, count#37] -(60) HashAggregate [codegen id : 13] +(60) HashAggregate [codegen id : 7] Input [2]: [segment#35, count#37] Keys [1]: [segment#35] Functions [1]: [count(1)] @@ -350,7 +350,7 @@ Arguments: 100, [segment#35 ASC NULLS FIRST, num_customers#39 ASC NULLS FIRST], Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (66) -+- * CometColumnarToRow (65) ++- CometNativeColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometNativeScan parquet spark_catalog.default.date_dim (62) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 12 Input [3]: [d_date_sk#17, d_year#41, d_moy#42] Arguments: [d_date_sk#17], [d_date_sk#17] -(65) CometColumnarToRow [codegen id : 1] +(65) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (66) BroadcastExchange @@ -382,7 +382,7 @@ Subquery:2 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometNativeScan parquet spark_catalog.default.date_dim (67) @@ -403,7 +403,7 @@ Condition : (((isnotnull(d_month_seq#43) AND (d_month_seq#43 >= Subquery scalar- Input [2]: [d_date_sk#31, d_month_seq#43] Arguments: [d_date_sk#31], [d_date_sk#31] -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [1]: [d_date_sk#31] (71) BroadcastExchange @@ -411,7 +411,7 @@ Input [1]: [d_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] Subquery:4 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* CometColumnarToRow (78) +CometNativeColumnarToRow (78) +- CometHashAggregate (77) +- CometExchange (76) +- CometHashAggregate (75) @@ -449,11 +449,11 @@ Input [1]: [(d_month_seq + 1)#51] Keys [1]: [(d_month_seq + 1)#51] Functions: [] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [1]: [(d_month_seq + 1)#51] Subquery:5 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* CometColumnarToRow (85) +CometNativeColumnarToRow (85) +- CometHashAggregate (84) +- CometExchange (83) +- CometHashAggregate (82) @@ -491,7 +491,7 @@ Input [1]: [(d_month_seq + 3)#55] Keys [1]: [(d_month_seq + 3)#55] Functions: [] -(85) CometColumnarToRow [codegen id : 1] +(85) CometNativeColumnarToRow Input [1]: [(d_month_seq + 3)#55] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt index 6ba04a3153..273efea475 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,17 +42,17 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange @@ -61,11 +61,11 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : :- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometExchange : : : : : +- CometHashAggregate @@ -73,7 +73,7 @@ TakeOrderedAndProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -82,21 +82,21 @@ TakeOrderedAndProject : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter : :- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -104,7 +104,7 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/simplified.txt index ba04de874d..3e9405f691 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [segment] #1 - WholeStageCodegen (12) + WholeStageCodegen (6) HashAggregate [segment] [count,count] HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_sk] #2 - WholeStageCodegen (11) + WholeStageCodegen (5) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -20,11 +20,11 @@ TakeOrderedAndProject [segment,num_customers,segment_base] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [c_customer_sk,c_current_addr_sk] CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [c_customer_sk,c_current_addr_sk] Project [c_customer_sk,c_current_addr_sk] BroadcastHashJoin [customer_sk,c_customer_sk] @@ -42,12 +42,10 @@ TakeOrderedAndProject [segment,num_customers,segment_base] Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (2) Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] Filter [ws_item_sk,ws_bill_customer_sk] @@ -57,71 +55,57 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ReusedSubquery [d_date_sk] #1 InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #9 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #10 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #3 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #9 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #10 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #11 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_county,ca_state] + CometFilter [ca_address_sk,ca_county,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] InputAdapter BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_county,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_county,s_state] + CometFilter [s_county,s_state] + CometNativeScan parquet spark_catalog.default.store [s_county,s_state] InputAdapter ReusedExchange [d_date_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/explain.txt index 26ce2a5731..8d2ba8f118 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/explain.txt @@ -337,7 +337,7 @@ Input [3]: [segment#40, num_customers#42, segment_base#43] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) @@ -358,7 +358,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (63) BroadcastExchange @@ -369,7 +369,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (68) -+- * CometColumnarToRow (67) ++- CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) @@ -390,7 +390,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= Subquery scalar- Input [2]: [d_date_sk#33, d_month_seq#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (68) BroadcastExchange @@ -398,7 +398,7 @@ Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:4 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* CometColumnarToRow (75) +CometNativeColumnarToRow (75) +- CometHashAggregate (74) +- CometExchange (73) +- CometHashAggregate (72) @@ -436,11 +436,11 @@ Input [1]: [(d_month_seq + 1)#47] Keys [1]: [(d_month_seq + 1)#47] Functions: [] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [(d_month_seq + 1)#47] Subquery:5 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* CometColumnarToRow (82) +CometNativeColumnarToRow (82) +- CometHashAggregate (81) +- CometExchange (80) +- CometHashAggregate (79) @@ -478,7 +478,7 @@ Input [1]: [(d_month_seq + 3)#51] Keys [1]: [(d_month_seq + 3)#51] Functions: [] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [(d_month_seq + 3)#51] Subquery:6 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt index 98fce92fb1..0b482769c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,11 +53,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : :- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometExchange : : : : : +- CometHashAggregate @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/simplified.txt index ae0cecb1ba..89c615df18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] @@ -57,32 +55,26 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #3 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 CometProject [ca_state] [ca_address_sk,ca_county,ca_state] CometFilter [ca_address_sk,ca_county,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt index 26ce2a5731..8d2ba8f118 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt @@ -337,7 +337,7 @@ Input [3]: [segment#40, num_customers#42, segment_base#43] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) @@ -358,7 +358,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (63) BroadcastExchange @@ -369,7 +369,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (68) -+- * CometColumnarToRow (67) ++- CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) @@ -390,7 +390,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= Subquery scalar- Input [2]: [d_date_sk#33, d_month_seq#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (68) BroadcastExchange @@ -398,7 +398,7 @@ Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:4 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* CometColumnarToRow (75) +CometNativeColumnarToRow (75) +- CometHashAggregate (74) +- CometExchange (73) +- CometHashAggregate (72) @@ -436,11 +436,11 @@ Input [1]: [(d_month_seq + 1)#47] Keys [1]: [(d_month_seq + 1)#47] Functions: [] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [(d_month_seq + 1)#47] Subquery:5 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* CometColumnarToRow (82) +CometNativeColumnarToRow (82) +- CometHashAggregate (81) +- CometExchange (80) +- CometHashAggregate (79) @@ -478,7 +478,7 @@ Input [1]: [(d_month_seq + 3)#51] Keys [1]: [(d_month_seq + 3)#51] Functions: [] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [(d_month_seq + 3)#51] Subquery:6 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/extended.txt index 98fce92fb1..0b482769c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,11 +53,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : :- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometExchange : : : : : +- CometHashAggregate @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate 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 ae0cecb1ba..89c615df18 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 @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] @@ -57,32 +55,26 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #3 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 CometProject [ca_state] [ca_address_sk,ca_county,ca_state] CometFilter [ca_address_sk,ca_county,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/explain.txt index 358eb94c69..23cbbb525e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [3]: [brand_id#13, brand#14, ext_price#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [3]: [brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/simplified.txt index a2adacc531..c31575cc36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_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 [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] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [brand_id,brand,ext_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 [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] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/explain.txt index 62396854a7..332b99e63f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (67) +- * HashAggregate (66) - +- * CometColumnarToRow (65) + +- CometNativeColumnarToRow (65) +- CometColumnarExchange (64) +- * HashAggregate (63) +- Union (62) :- * HashAggregate (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometColumnarExchange (27) : +- * HashAggregate (26) : +- * Project (25) @@ -20,12 +20,12 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometProject (21) : +- CometBroadcastHashJoin (20) : :- CometFilter (15) @@ -35,7 +35,7 @@ TakeOrderedAndProject (67) : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.item (16) :- * HashAggregate (45) - : +- * CometColumnarToRow (44) + : +- CometNativeColumnarToRow (44) : +- CometColumnarExchange (43) : +- * HashAggregate (42) : +- * Project (41) @@ -51,7 +51,7 @@ TakeOrderedAndProject (67) : : +- ReusedExchange (36) : +- ReusedExchange (39) +- * HashAggregate (61) - +- * CometColumnarToRow (60) + +- CometNativeColumnarToRow (60) +- CometColumnarExchange (59) +- * HashAggregate (58) +- * Project (57) @@ -76,23 +76,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] @@ -111,20 +111,20 @@ Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isno Input [2]: [ca_address_sk#7, ca_gmt_offset#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [ca_address_sk#7] (11) BroadcastExchange Input [1]: [ca_address_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] @@ -166,24 +166,24 @@ Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCod Input [2]: [i_item_sk#9, i_item_id#10] Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true) AS i_item_id#14] -(22) CometColumnarToRow [codegen id : 3] +(22) CometNativeColumnarToRow Input [2]: [i_item_sk#9, i_item_id#14] (23) BroadcastExchange Input [2]: [i_item_sk#9, i_item_id#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 4] +(24) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#9] Join type: Inner Join condition: None -(25) Project [codegen id : 4] +(25) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#3, i_item_id#14] Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] -(26) HashAggregate [codegen id : 4] +(26) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#14] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -194,10 +194,10 @@ Results [2]: [i_item_id#14, sum#16] Input [2]: [i_item_id#14, sum#16] Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [2]: [i_item_id#14, sum#16] -(29) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 2] Input [2]: [i_item_id#14, sum#16] Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -212,53 +212,53 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 9] +(31) ColumnarToRow [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -(32) Filter [codegen id : 9] +(32) Filter [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) (33) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#24] -(34) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#23] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(35) Project [codegen id : 9] +(35) Project [codegen id : 3] Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] (36) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#25] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_addr_sk#20] Right keys [1]: [ca_address_sk#25] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 3] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] (39) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#26, i_item_id#27] -(40) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#21] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(41) Project [codegen id : 3] Output [2]: [cs_ext_sales_price#22, i_item_id#27] Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 3] Input [2]: [cs_ext_sales_price#22, i_item_id#27] Keys [1]: [i_item_id#27] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] @@ -269,10 +269,10 @@ Results [2]: [i_item_id#27, sum#29] Input [2]: [i_item_id#27, sum#29] Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometColumnarToRow [codegen id : 10] +(44) CometNativeColumnarToRow Input [2]: [i_item_id#27, sum#29] -(45) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 4] Input [2]: [i_item_id#27, sum#29] Keys [1]: [i_item_id#27] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] @@ -287,53 +287,53 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 14] +(47) ColumnarToRow [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -(48) Filter [codegen id : 14] +(48) Filter [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) (49) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#36] -(50) BroadcastHashJoin [codegen id : 14] +(50) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#35] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(51) Project [codegen id : 14] +(51) Project [codegen id : 5] Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] (52) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#37] -(53) BroadcastHashJoin [codegen id : 14] +(53) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_bill_addr_sk#33] Right keys [1]: [ca_address_sk#37] Join type: Inner Join condition: None -(54) Project [codegen id : 14] +(54) Project [codegen id : 5] Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] (55) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#38, i_item_id#39] -(56) BroadcastHashJoin [codegen id : 14] +(56) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_item_sk#32] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(57) Project [codegen id : 5] Output [2]: [ws_ext_sales_price#34, i_item_id#39] Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] -(58) HashAggregate [codegen id : 14] +(58) HashAggregate [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#39] Keys [1]: [i_item_id#39] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] @@ -344,10 +344,10 @@ Results [2]: [i_item_id#39, sum#41] Input [2]: [i_item_id#39, sum#41] Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(60) CometColumnarToRow [codegen id : 15] +(60) CometNativeColumnarToRow Input [2]: [i_item_id#39, sum#41] -(61) HashAggregate [codegen id : 15] +(61) HashAggregate [codegen id : 6] Input [2]: [i_item_id#39, sum#41] Keys [1]: [i_item_id#39] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] @@ -356,7 +356,7 @@ Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34) (62) Union -(63) HashAggregate [codegen id : 16] +(63) HashAggregate [codegen id : 7] Input [2]: [i_item_id#18, total_sales#19] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(total_sales#19)] @@ -367,10 +367,10 @@ Results [3]: [i_item_id#18, sum#46, isEmpty#47] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(65) CometColumnarToRow [codegen id : 17] +(65) CometNativeColumnarToRow Input [3]: [i_item_id#18, sum#46, isEmpty#47] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 8] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Keys [1]: [i_item_id#18] Functions [1]: [sum(total_sales#19)] @@ -385,7 +385,7 @@ Arguments: 100, [total_sales#49 ASC NULLS FIRST], [i_item_id#18, total_sales#49] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometNativeScan parquet spark_catalog.default.date_dim (68) @@ -406,7 +406,7 @@ Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2 Input [3]: [d_date_sk#6, d_year#50, d_moy#51] Arguments: [d_date_sk#6], [d_date_sk#6] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (72) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt index 7a2832757d..ad055b48b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,22 +19,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -44,7 +44,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,17 +58,17 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -78,7 +78,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -92,17 +92,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/simplified.txt index 81d90bdfb2..4d9a45fc72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -27,41 +27,35 @@ TakeOrderedAndProject [total_sales,i_item_id] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (10) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #6 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_color] + CometNativeScan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -80,12 +74,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ReusedExchange [ca_address_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] BroadcastHashJoin [ws_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/explain.txt index 5562e86b91..7f18d8071c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/simplified.txt index 19eca782cd..eb64094749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt index 5562e86b91..7f18d8071c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 19eca782cd..eb64094749 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/explain.txt index c3ba92766c..b63efdd787 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.call_center (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.catalog_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [2]: [cc_call_center_sk#14, cc_name#15] (17) BroadcastExchange Input [2]: [cc_call_center_sk#14, cc_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_call_center_sk#6] Right keys [1]: [cc_call_center_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] @@ -151,10 +151,10 @@ Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] (27) Window Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] (32) ReusedExchange [Reuses operator id: 21] Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQU Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] (38) Window Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] @@ -228,13 +228,13 @@ Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales# Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] @@ -245,14 +245,14 @@ Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] (46) Window Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] @@ -260,13 +260,13 @@ Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales# Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt index 65c66a7da8..66c510aaa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/simplified.txt index 4f9ac35f4d..c085ecfb51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [cs_item_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [cc_call_center_sk,cc_name] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/explain.txt index eb21a3abd7..24505a9158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/simplified.txt index b854e818db..2f01d9b13f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt index eb21a3abd7..24505a9158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 b854e818db..2f01d9b13f 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/explain.txt index 34406110bc..89edbd5aaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (53) : +- * BroadcastHashJoin Inner BuildRight (34) : :- * Filter (18) : : +- * HashAggregate (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- * HashAggregate (14) : : +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (53) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -25,7 +25,7 @@ TakeOrderedAndProject (53) : +- BroadcastExchange (33) : +- * Filter (32) : +- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * Project (27) @@ -40,7 +40,7 @@ TakeOrderedAndProject (53) +- BroadcastExchange (50) +- * Filter (49) +- * HashAggregate (48) - +- * CometColumnarToRow (47) + +- CometNativeColumnarToRow (47) +- CometColumnarExchange (46) +- * HashAggregate (45) +- * Project (44) @@ -62,10 +62,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) @@ -84,37 +84,37 @@ Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache. Input [2]: [i_item_sk#5, i_item_id#6] Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [i_item_sk#5, i_item_id#7] (8) BroadcastExchange Input [2]: [i_item_sk#5, i_item_id#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] (11) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#8] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#2, i_item_id#7] Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#2, i_item_id#7] Keys [1]: [i_item_id#7] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -125,17 +125,17 @@ Results [2]: [i_item_id#7, sum#10] Input [2]: [i_item_id#7, sum#10] Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 12] +(16) CometNativeColumnarToRow Input [2]: [i_item_id#7, sum#10] -(17) HashAggregate [codegen id : 12] +(17) HashAggregate [codegen id : 6] Input [2]: [i_item_id#7, sum#10] Keys [1]: [i_item_id#7] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] Results [2]: [i_item_id#7 AS item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS ss_item_rev#13] -(18) Filter [codegen id : 12] +(18) Filter [codegen id : 6] Input [2]: [item_id#12, ss_item_rev#13] Condition : isnotnull(ss_item_rev#13) @@ -147,40 +147,40 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 2] Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -(21) Filter [codegen id : 6] +(21) Filter [codegen id : 2] Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] Condition : isnotnull(cs_item_sk#14) (22) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#17, i_item_id#18] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_item_sk#14] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(24) Project [codegen id : 2] Output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#18] Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#17, i_item_id#18] (25) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#19] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 2] Output [2]: [cs_ext_sales_price#15, i_item_id#18] Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#18, d_date_sk#19] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [2]: [cs_ext_sales_price#15, i_item_id#18] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] @@ -191,17 +191,17 @@ Results [2]: [i_item_id#18, sum#21] Input [2]: [i_item_id#18, sum#21] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [2]: [i_item_id#18, sum#21] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [2]: [i_item_id#18, sum#21] Keys [1]: [i_item_id#18] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#15))#22] Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#15))#22,17,2) AS cs_item_rev#24] -(32) Filter [codegen id : 7] +(32) Filter [codegen id : 3] Input [2]: [item_id#23, cs_item_rev#24] Condition : isnotnull(cs_item_rev#24) @@ -209,13 +209,13 @@ Condition : isnotnull(cs_item_rev#24) Input [2]: [item_id#23, cs_item_rev#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 12] +(34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#23] Join type: Inner Join condition: ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) -(35) Project [codegen id : 12] +(35) Project [codegen id : 6] Output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#24] Input [4]: [item_id#12, ss_item_rev#13, item_id#23, cs_item_rev#24] @@ -227,40 +227,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 4] Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -(38) Filter [codegen id : 10] +(38) Filter [codegen id : 4] Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] Condition : isnotnull(ws_item_sk#25) (39) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#28, i_item_id#29] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#25] Right keys [1]: [i_item_sk#28] Join type: Inner Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 4] Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#29] Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#28, i_item_id#29] (42) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#30] -(43) BroadcastHashJoin [codegen id : 10] +(43) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#27] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(44) Project [codegen id : 10] +(44) Project [codegen id : 4] Output [2]: [ws_ext_sales_price#26, i_item_id#29] Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#29, d_date_sk#30] -(45) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 4] Input [2]: [ws_ext_sales_price#26, i_item_id#29] Keys [1]: [i_item_id#29] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] @@ -271,17 +271,17 @@ Results [2]: [i_item_id#29, sum#32] Input [2]: [i_item_id#29, sum#32] Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(47) CometColumnarToRow [codegen id : 11] +(47) CometNativeColumnarToRow Input [2]: [i_item_id#29, sum#32] -(48) HashAggregate [codegen id : 11] +(48) HashAggregate [codegen id : 5] Input [2]: [i_item_id#29, sum#32] Keys [1]: [i_item_id#29] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#33] Results [2]: [i_item_id#29 AS item_id#34, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#33,17,2) AS ws_item_rev#35] -(49) Filter [codegen id : 11] +(49) Filter [codegen id : 5] Input [2]: [item_id#34, ws_item_rev#35] Condition : isnotnull(ws_item_rev#35) @@ -289,13 +289,13 @@ Condition : isnotnull(ws_item_rev#35) Input [2]: [item_id#34, ws_item_rev#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(51) BroadcastHashJoin [codegen id : 12] +(51) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#34] Join type: Inner Join condition: ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#35)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) -(52) Project [codegen id : 12] +(52) Project [codegen id : 6] Output [8]: [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ss_dev#36, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS cs_dev#37, ws_item_rev#35, (((ws_item_rev#35 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ws_dev#38, (((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35) / 3) AS average#39] Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#24, item_id#34, ws_item_rev#35] @@ -307,7 +307,7 @@ Arguments: 100, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometFilter (55) @@ -357,7 +357,7 @@ Arguments: [d_date#40], [d_date#41], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#40] Arguments: [d_date_sk#8], [d_date_sk#8] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (63) BroadcastExchange @@ -365,7 +365,7 @@ Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 57 Hosting Expression = Subquery scalar-subquery#43, [id=#44] -* CometColumnarToRow (67) +CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometNativeScan parquet spark_catalog.default.date_dim (64) @@ -386,7 +386,7 @@ Condition : (isnotnull(d_date#45) AND (d_date#45 = 2000-01-03)) Input [2]: [d_date#45, d_week_seq#46] Arguments: [d_week_seq#46], [d_week_seq#46] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_week_seq#46] Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt index 4b333b4991..ff5ea85d39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -26,18 +26,18 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -54,7 +54,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -66,12 +66,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -88,7 +88,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -100,12 +100,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter @@ -114,7 +114,7 @@ TakeOrderedAndProject +- CometProject +- CometFilter : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/simplified.txt index 94adc941c8..7ce9db43e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/simplified.txt @@ -1,15 +1,15 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (12) + WholeStageCodegen (6) Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] Project [item_id,ss_item_rev,cs_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] Filter [ss_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -21,43 +21,37 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (3) Filter [cs_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -74,13 +68,13 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) + WholeStageCodegen (5) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (10) + WholeStageCodegen (4) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/explain.txt index 0307d0df0e..3254e2470b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/explain.txt @@ -292,7 +292,7 @@ Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometFilter (54) @@ -342,7 +342,7 @@ Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (62) BroadcastExchange @@ -350,7 +350,7 @@ Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* CometColumnarToRow (66) +CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) Input [2]: [d_date#41, d_week_seq#42] Arguments: [d_week_seq#42], [d_week_seq#42] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_week_seq#42] Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt index 3d3dd68990..547d7002b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -25,7 +25,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -43,7 +43,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -73,7 +73,7 @@ CometColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,7 +103,7 @@ CometColumnarToRow +- CometProject +- CometFilter : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/simplified.txt index 96e11bc0c7..1dfbd06b10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/simplified.txt @@ -18,24 +18,20 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt index 0307d0df0e..3254e2470b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt @@ -292,7 +292,7 @@ Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometFilter (54) @@ -342,7 +342,7 @@ Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (62) BroadcastExchange @@ -350,7 +350,7 @@ Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* CometColumnarToRow (66) +CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) Input [2]: [d_date#41, d_week_seq#42] Arguments: [d_week_seq#42], [d_week_seq#42] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_week_seq#42] Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/extended.txt index 3d3dd68990..547d7002b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -25,7 +25,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -43,7 +43,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -73,7 +73,7 @@ CometColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,7 +103,7 @@ CometColumnarToRow +- CometProject +- CometFilter : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 96e11bc0c7..1dfbd06b10 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 @@ -18,24 +18,20 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/explain.txt index 1bb4b7dc8f..4f489ee77b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (42) +CometNativeColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -234,6 +234,6 @@ Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_ Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#69,(mon_sales1 / mon_sales2)#70,(tue_sales1 / tue_sales2)#71,(wed_sales1 / wed_sales2)#72,(thu_sales1 / thu_sales2)#73,(fri_sales1 / fri_sales2)#74,(sat_sales1 / sat_sales2)#75]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] -(42) CometColumnarToRow [codegen id : 1] +(42) CometNativeColumnarToRow Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/extended.txt index bfd467b72b..e23616d099 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/simplified.txt index 72823b14c1..16c54015b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/simplified.txt @@ -1,44 +1,42 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [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_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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [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] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] +CometNativeColumnarToRow + CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] + CometHashAggregate [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_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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] + CometHashAggregate [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] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [d_week_seq] #7 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/explain.txt index 93d8a46c2d..46fa14002c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (40) +- * Filter (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * Project (34) @@ -11,7 +11,7 @@ TakeOrderedAndProject (40) : +- * BroadcastHashJoin Inner BuildRight (17) : :- * Project (15) : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) + : : :- CometNativeColumnarToRow (9) : : : +- CometProject (8) : : : +- CometBroadcastHashJoin (7) : : : :- CometProject (3) @@ -26,7 +26,7 @@ TakeOrderedAndProject (40) : : +- Scan parquet spark_catalog.default.store_sales (10) : +- ReusedExchange (16) +- BroadcastExchange (32) - +- * CometColumnarToRow (31) + +- CometNativeColumnarToRow (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (20) @@ -80,7 +80,7 @@ Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(9) CometColumnarToRow [codegen id : 4] +(9) CometNativeColumnarToRow Input [2]: [ca_state#3, c_customer_sk#4] (10) Scan parquet spark_catalog.default.store_sales @@ -102,26 +102,26 @@ Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#4] Right keys [1]: [ss_customer_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] (16) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 2] Output [2]: [ca_state#3, ss_item_sk#6] Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] @@ -181,24 +181,24 @@ Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCod Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] Arguments: [i_item_sk#11], [i_item_sk#11] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [i_item_sk#11] (32) BroadcastExchange Input [1]: [i_item_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 4] +(33) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#6] Right keys [1]: [i_item_sk#11] Join type: Inner Join condition: None -(34) Project [codegen id : 4] +(34) Project [codegen id : 2] Output [1]: [ca_state#3] Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] -(35) HashAggregate [codegen id : 4] +(35) HashAggregate [codegen id : 2] Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] @@ -209,17 +209,17 @@ Results [2]: [ca_state#3, count#21] Input [2]: [ca_state#3, count#21] Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometColumnarToRow [codegen id : 5] +(37) CometNativeColumnarToRow Input [2]: [ca_state#3, count#21] -(38) HashAggregate [codegen id : 5] +(38) HashAggregate [codegen id : 3] Input [2]: [ca_state#3, count#21] Keys [1]: [ca_state#3] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] Results [2]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24] -(39) Filter [codegen id : 5] +(39) Filter [codegen id : 3] Input [2]: [state#23, cnt#24] Condition : (cnt#24 >= 10) @@ -231,7 +231,7 @@ Arguments: 100, [cnt#24 ASC NULLS FIRST], [state#23, cnt#24] Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#25] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#28] Keys [1]: [d_month_seq#28] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt index fe3b28849a..f2cdb50e03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometProject @@ -25,11 +25,11 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -38,11 +38,11 @@ TakeOrderedAndProject : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -51,7 +51,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/simplified.txt index cec4eecbc6..fed4bb3845 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) + WholeStageCodegen (3) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -13,8 +13,8 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ca_state,ss_item_sk,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ca_state,c_customer_sk] CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] CometProject [ca_state] [ca_address_sk,ca_state] @@ -32,38 +32,32 @@ TakeOrderedAndProject [cnt,state] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #7 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #8 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_current_price,i_category] + CometFilter [i_current_price,i_category] + CometNativeScan 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/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/explain.txt index da1b420416..a2f9447ab6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#24, cnt#25] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#26] Keys [1]: [d_month_seq#26] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#26] Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt index f5b69fc6cf..20115b61c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt @@ -23,11 +23,11 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/simplified.txt index 2c772b5998..2f287bd993 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/simplified.txt @@ -25,22 +25,18 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt index da1b420416..7aaf2d1244 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#24, cnt#25] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -270,36 +270,36 @@ Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer (46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] +Output [3]: [d_month_seq#13, d_year#26, d_moy#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) +Input [3]: [d_month_seq#13, d_year#26, d_moy#27] +Condition : (((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2000)) AND (d_moy#27 = 1)) (48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] +Input [3]: [d_month_seq#13, d_year#26, d_moy#27] +Arguments: [d_month_seq#13], [d_month_seq#13] (49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +Input [1]: [d_month_seq#13] +Keys [1]: [d_month_seq#13] Functions: [] (50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Input [1]: [d_month_seq#13] +Arguments: hashpartitioning(d_month_seq#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] (51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] +Input [1]: [d_month_seq#13] +Keys [1]: [d_month_seq#13] Functions: [] -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] +(52) CometNativeColumnarToRow +Input [1]: [d_month_seq#13] Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/extended.txt index f5b69fc6cf..20115b61c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/extended.txt @@ -23,11 +23,11 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate 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 2c772b5998..2f287bd993 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 @@ -25,22 +25,18 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/explain.txt index 5584119109..573325eb66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (67) +- * HashAggregate (66) - +- * CometColumnarToRow (65) + +- CometNativeColumnarToRow (65) +- CometColumnarExchange (64) +- * HashAggregate (63) +- Union (62) :- * HashAggregate (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometColumnarExchange (27) : +- * HashAggregate (26) : +- * Project (25) @@ -20,12 +20,12 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometProject (21) : +- CometBroadcastHashJoin (20) : :- CometFilter (15) @@ -35,7 +35,7 @@ TakeOrderedAndProject (67) : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.item (16) :- * HashAggregate (45) - : +- * CometColumnarToRow (44) + : +- CometNativeColumnarToRow (44) : +- CometColumnarExchange (43) : +- * HashAggregate (42) : +- * Project (41) @@ -51,7 +51,7 @@ TakeOrderedAndProject (67) : : +- ReusedExchange (36) : +- ReusedExchange (39) +- * HashAggregate (61) - +- * CometColumnarToRow (60) + +- CometNativeColumnarToRow (60) +- CometColumnarExchange (59) +- * HashAggregate (58) +- * Project (57) @@ -76,23 +76,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] @@ -111,20 +111,20 @@ Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isno Input [2]: [ca_address_sk#7, ca_gmt_offset#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [ca_address_sk#7] (11) BroadcastExchange Input [1]: [ca_address_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] @@ -166,24 +166,24 @@ Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCod Input [2]: [i_item_sk#9, i_item_id#10] Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true) AS i_item_id#14] -(22) CometColumnarToRow [codegen id : 3] +(22) CometNativeColumnarToRow Input [2]: [i_item_sk#9, i_item_id#14] (23) BroadcastExchange Input [2]: [i_item_sk#9, i_item_id#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 4] +(24) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#9] Join type: Inner Join condition: None -(25) Project [codegen id : 4] +(25) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#3, i_item_id#14] Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] -(26) HashAggregate [codegen id : 4] +(26) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#14] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -194,10 +194,10 @@ Results [2]: [i_item_id#14, sum#16] Input [2]: [i_item_id#14, sum#16] Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [2]: [i_item_id#14, sum#16] -(29) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 2] Input [2]: [i_item_id#14, sum#16] Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -212,53 +212,53 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 9] +(31) ColumnarToRow [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -(32) Filter [codegen id : 9] +(32) Filter [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) (33) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#24] -(34) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#23] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(35) Project [codegen id : 9] +(35) Project [codegen id : 3] Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] (36) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#25] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_addr_sk#20] Right keys [1]: [ca_address_sk#25] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 3] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] (39) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#26, i_item_id#27] -(40) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#21] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(41) Project [codegen id : 3] Output [2]: [cs_ext_sales_price#22, i_item_id#27] Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 3] Input [2]: [cs_ext_sales_price#22, i_item_id#27] Keys [1]: [i_item_id#27] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] @@ -269,10 +269,10 @@ Results [2]: [i_item_id#27, sum#29] Input [2]: [i_item_id#27, sum#29] Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometColumnarToRow [codegen id : 10] +(44) CometNativeColumnarToRow Input [2]: [i_item_id#27, sum#29] -(45) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 4] Input [2]: [i_item_id#27, sum#29] Keys [1]: [i_item_id#27] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] @@ -287,53 +287,53 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 14] +(47) ColumnarToRow [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -(48) Filter [codegen id : 14] +(48) Filter [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) (49) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#36] -(50) BroadcastHashJoin [codegen id : 14] +(50) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#35] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(51) Project [codegen id : 14] +(51) Project [codegen id : 5] Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] (52) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#37] -(53) BroadcastHashJoin [codegen id : 14] +(53) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_bill_addr_sk#33] Right keys [1]: [ca_address_sk#37] Join type: Inner Join condition: None -(54) Project [codegen id : 14] +(54) Project [codegen id : 5] Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] (55) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#38, i_item_id#39] -(56) BroadcastHashJoin [codegen id : 14] +(56) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_item_sk#32] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(57) Project [codegen id : 5] Output [2]: [ws_ext_sales_price#34, i_item_id#39] Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] -(58) HashAggregate [codegen id : 14] +(58) HashAggregate [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#39] Keys [1]: [i_item_id#39] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] @@ -344,10 +344,10 @@ Results [2]: [i_item_id#39, sum#41] Input [2]: [i_item_id#39, sum#41] Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(60) CometColumnarToRow [codegen id : 15] +(60) CometNativeColumnarToRow Input [2]: [i_item_id#39, sum#41] -(61) HashAggregate [codegen id : 15] +(61) HashAggregate [codegen id : 6] Input [2]: [i_item_id#39, sum#41] Keys [1]: [i_item_id#39] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] @@ -356,7 +356,7 @@ Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34) (62) Union -(63) HashAggregate [codegen id : 16] +(63) HashAggregate [codegen id : 7] Input [2]: [i_item_id#18, total_sales#19] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(total_sales#19)] @@ -367,10 +367,10 @@ Results [3]: [i_item_id#18, sum#46, isEmpty#47] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(65) CometColumnarToRow [codegen id : 17] +(65) CometNativeColumnarToRow Input [3]: [i_item_id#18, sum#46, isEmpty#47] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 8] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Keys [1]: [i_item_id#18] Functions [1]: [sum(total_sales#19)] @@ -385,7 +385,7 @@ Arguments: 100, [i_item_id#18 ASC NULLS FIRST, total_sales#49 ASC NULLS FIRST], Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometNativeScan parquet spark_catalog.default.date_dim (68) @@ -406,7 +406,7 @@ Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 1 Input [3]: [d_date_sk#6, d_year#50, d_moy#51] Arguments: [d_date_sk#6], [d_date_sk#6] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (72) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt index 7a2832757d..ad055b48b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,22 +19,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -44,7 +44,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,17 +58,17 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -78,7 +78,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -92,17 +92,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/simplified.txt index 4c05038b34..56a5af7cf7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -27,41 +27,35 @@ TakeOrderedAndProject [i_item_id,total_sales] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (10) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #6 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -80,12 +74,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ReusedExchange [ca_address_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] BroadcastHashJoin [ws_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/explain.txt index e5e165b5ec..0c43a7c934 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/simplified.txt index 086da3b776..92a33e24ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt index e5e165b5ec..0c43a7c934 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 086da3b776..92a33e24ce 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/explain.txt index e50623a19c..34d53d4d65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/explain.txt @@ -2,7 +2,7 @@ * Project (69) +- * BroadcastNestedLoopJoin Inner BuildRight (68) :- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -21,33 +21,33 @@ : : : : : : : +- * ColumnarToRow (2) : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) + : : : : : : +- CometNativeColumnarToRow (7) : : : : : : +- CometProject (6) : : : : : : +- CometFilter (5) : : : : : : +- CometNativeScan parquet spark_catalog.default.store (4) : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) + : : : : : +- CometNativeColumnarToRow (14) : : : : : +- CometProject (13) : : : : : +- CometFilter (12) : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (11) : : : : +- ReusedExchange (18) : : : +- BroadcastExchange (24) - : : : +- * CometColumnarToRow (23) + : : : +- CometNativeColumnarToRow (23) : : : +- CometFilter (22) : : : +- CometNativeScan parquet spark_catalog.default.customer (21) : : +- BroadcastExchange (31) - : : +- * CometColumnarToRow (30) + : : +- CometNativeColumnarToRow (30) : : +- CometProject (29) : : +- CometFilter (28) : : +- CometNativeScan parquet spark_catalog.default.customer_address (27) : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometProject (36) : +- CometFilter (35) : +- CometNativeScan parquet spark_catalog.default.item (34) +- BroadcastExchange (67) +- * HashAggregate (66) - +- * CometColumnarToRow (65) + +- CometNativeColumnarToRow (65) +- CometColumnarExchange (64) +- * HashAggregate (63) +- * Project (62) @@ -78,10 +78,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) @@ -100,20 +100,20 @@ Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotn Input [2]: [s_store_sk#8, s_gmt_offset#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (8) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 1] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] @@ -132,33 +132,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] Arguments: [p_promo_sk#10], [p_promo_sk#10] -(14) CometColumnarToRow [codegen id : 2] +(14) CometNativeColumnarToRow Input [1]: [p_promo_sk#10] (15) BroadcastExchange Input [1]: [p_promo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 7] +(16) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_promo_sk#4] Right keys [1]: [p_promo_sk#10] Join type: Inner Join condition: None -(17) Project [codegen id : 7] +(17) Project [codegen id : 1] Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] (18) ReusedExchange [Reuses operator id: 74] Output [1]: [d_date_sk#14] -(19) BroadcastHashJoin [codegen id : 7] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 7] +(20) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] @@ -173,20 +173,20 @@ ReadSchema: struct Input [2]: [c_customer_sk#15, c_current_addr_sk#16] Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [c_customer_sk#15, c_current_addr_sk#16] (24) BroadcastExchange Input [2]: [c_customer_sk#15, c_current_addr_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 7] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 7] +(26) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#15, c_current_addr_sk#16] @@ -205,20 +205,20 @@ Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -5.00)) AND is Input [2]: [ca_address_sk#17, ca_gmt_offset#18] Arguments: [ca_address_sk#17], [ca_address_sk#17] -(30) CometColumnarToRow [codegen id : 5] +(30) CometNativeColumnarToRow Input [1]: [ca_address_sk#17] (31) BroadcastExchange Input [1]: [ca_address_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#16] Right keys [1]: [ca_address_sk#17] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16, ca_address_sk#17] @@ -237,24 +237,24 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [i_item_sk#19, i_category#20] Arguments: [i_item_sk#19], [i_item_sk#19] -(37) CometColumnarToRow [codegen id : 6] +(37) CometNativeColumnarToRow Input [1]: [i_item_sk#19] (38) BroadcastExchange Input [1]: [i_item_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 7] +(39) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#19] Join type: Inner Join condition: None -(40) Project [codegen id : 7] +(40) Project [codegen id : 1] Output [1]: [ss_ext_sales_price#5] Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#19] -(41) HashAggregate [codegen id : 7] +(41) HashAggregate [codegen id : 1] Input [1]: [ss_ext_sales_price#5] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] @@ -265,10 +265,10 @@ Results [1]: [sum#22] Input [1]: [sum#22] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 15] +(43) CometNativeColumnarToRow Input [1]: [sum#22] -(44) HashAggregate [codegen id : 15] +(44) HashAggregate [codegen id : 4] Input [1]: [sum#22] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] @@ -283,79 +283,79 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 13] +(46) ColumnarToRow [codegen id : 2] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -(47) Filter [codegen id : 13] +(47) Filter [codegen id : 2] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [s_store_sk#30] -(49) BroadcastHashJoin [codegen id : 13] +(49) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#27] Right keys [1]: [s_store_sk#30] Join type: Inner Join condition: None -(50) Project [codegen id : 13] +(50) Project [codegen id : 2] Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#30] (51) ReusedExchange [Reuses operator id: 74] Output [1]: [d_date_sk#31] -(52) BroadcastHashJoin [codegen id : 13] +(52) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#29] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(53) Project [codegen id : 13] +(53) Project [codegen id : 2] Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] (54) ReusedExchange [Reuses operator id: 24] Output [2]: [c_customer_sk#32, c_current_addr_sk#33] -(55) BroadcastHashJoin [codegen id : 13] +(55) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(56) Project [codegen id : 13] +(56) Project [codegen id : 2] Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#32, c_current_addr_sk#33] (57) ReusedExchange [Reuses operator id: 31] Output [1]: [ca_address_sk#34] -(58) BroadcastHashJoin [codegen id : 13] +(58) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_current_addr_sk#33] Right keys [1]: [ca_address_sk#34] Join type: Inner Join condition: None -(59) Project [codegen id : 13] +(59) Project [codegen id : 2] Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33, ca_address_sk#34] (60) ReusedExchange [Reuses operator id: 38] Output [1]: [i_item_sk#35] -(61) BroadcastHashJoin [codegen id : 13] +(61) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#25] Right keys [1]: [i_item_sk#35] Join type: Inner Join condition: None -(62) Project [codegen id : 13] +(62) Project [codegen id : 2] Output [1]: [ss_ext_sales_price#28] Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#35] -(63) HashAggregate [codegen id : 13] +(63) HashAggregate [codegen id : 2] Input [1]: [ss_ext_sales_price#28] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] @@ -366,10 +366,10 @@ Results [1]: [sum#37] Input [1]: [sum#37] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(65) CometColumnarToRow [codegen id : 14] +(65) CometNativeColumnarToRow Input [1]: [sum#37] -(66) HashAggregate [codegen id : 14] +(66) HashAggregate [codegen id : 3] Input [1]: [sum#37] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] @@ -380,11 +380,11 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#38,17,2) AS Input [1]: [total#39] Arguments: IdentityBroadcastMode, [plan_id=8] -(68) BroadcastNestedLoopJoin [codegen id : 15] +(68) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(69) Project [codegen id : 15] +(69) Project [codegen id : 4] Output [3]: [promotions#24, total#39, ((cast(promotions#24 as decimal(15,4)) / cast(total#39 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] Input [2]: [promotions#24, total#39] @@ -392,7 +392,7 @@ Input [2]: [promotions#24, total#39] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometProject (72) +- CometFilter (71) +- CometNativeScan parquet spark_catalog.default.date_dim (70) @@ -413,7 +413,7 @@ Condition : ((((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1 Input [3]: [d_date_sk#14, d_year#41, d_moy#42] Arguments: [d_date_sk#14], [d_date_sk#14] -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (74) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt index 0443f5fc55..025280d4ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt @@ -1,7 +1,7 @@ Project +- BroadcastNestedLoopJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,42 +21,42 @@ Project : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.promotion : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -74,26 +74,26 @@ Project : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/simplified.txt index f5bacac9c6..92507e130e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ -WholeStageCodegen (15) +WholeStageCodegen (4) Project [promotions,total] BroadcastNestedLoopJoin HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -25,61 +25,49 @@ WholeStageCodegen (15) Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometNativeColumnarToRow + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #8 - WholeStageCodegen (14) + WholeStageCodegen (3) HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (13) + WholeStageCodegen (2) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/explain.txt index 4f090653c8..29348cf7c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/explain.txt @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt index 7bae7d5724..f359e64d7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ Project : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/simplified.txt index 6ebe25b753..1709804080 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ WholeStageCodegen (2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] CometFilter [s_store_sk,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt index 4f090653c8..29348cf7c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt index 7bae7d5724..f359e64d7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt @@ -20,7 +20,7 @@ Project : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6ebe25b753..1709804080 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 @@ -22,12 +22,10 @@ WholeStageCodegen (2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] CometFilter [s_store_sk,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/explain.txt index 1d9748055e..fc3827793a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (29) +CometNativeColumnarToRow (29) +- CometTakeOrderedAndProject (28) +- CometHashAggregate (27) +- CometExchange (26) @@ -163,6 +163,6 @@ Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt index 52d7828b06..dc88244743 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/simplified.txt index de80c17349..1407717d50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/simplified.txt @@ -1,31 +1,29 @@ -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 [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 [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] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +CometNativeColumnarToRow + 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 [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 [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] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometProject [sm_type] [sm_ship_mode_sk,sm_type] + CometFilter [sm_ship_mode_sk,sm_type] + CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/explain.txt index 32f70c3d33..d9e5498574 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (30) +- * Project (29) +- * Filter (28) +- Window (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -16,7 +16,7 @@ TakeOrderedAndProject (30) : +- * BroadcastHashJoin Inner BuildRight (12) : :- * Project (10) : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject (30) : : +- Scan parquet spark_catalog.default.store_sales (5) : +- ReusedExchange (11) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -46,7 +46,7 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [2]: [i_item_sk#1, i_manager_id#5] (5) Scan parquet spark_catalog.default.store_sales @@ -68,26 +68,26 @@ Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] (11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#15, d_moy#16] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#13] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16] Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#16] @@ -102,24 +102,24 @@ ReadSchema: struct Input [1]: [s_store_sk#17] Condition : isnotnull(s_store_sk#17) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [s_store_sk#17] (17) BroadcastExchange Input [1]: [s_store_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#11] Right keys [1]: [s_store_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16, s_store_sk#17] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] Keys [2]: [i_manager_id#5, d_moy#16] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] @@ -130,10 +130,10 @@ Results [3]: [i_manager_id#5, d_moy#16, sum#19] Input [3]: [i_manager_id#5, d_moy#16, sum#19] Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [3]: [i_manager_id#5, d_moy#16, sum#19] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [3]: [i_manager_id#5, d_moy#16, sum#19] Keys [2]: [i_manager_id#5, d_moy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] @@ -148,18 +148,18 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumn Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] (27) Window Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 7] +(29) Project [codegen id : 4] Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] @@ -171,7 +171,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -192,7 +192,7 @@ Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_moy#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt index 0b6c2edaa0..5c7a26057d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject +- Project +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -25,17 +25,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/simplified.txt index 2933149226..7bb18e1026 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/simplified.txt @@ -1,53 +1,47 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manager_id,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometNativeColumnarToRow + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + WholeStageCodegen (3) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_manager_id,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/explain.txt index e2a0cceb1a..2c40f0d826 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/simplified.txt index 02166879a9..2f7b6145a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt index e2a0cceb1a..2c40f0d826 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 02166879a9..2f7b6145a3 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/explain.txt index c021738ca6..cc6e3125df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (180) +CometNativeColumnarToRow (180) +- CometSort (179) +- CometExchange (178) +- CometProject (177) @@ -48,7 +48,7 @@ : : : : : : : : : : : : : : : : : : +- * Filter (3) : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (8) : : : : : : : : : : : : : : : : : +- CometProject (7) : : : : : : : : : : : : : : : : : +- CometFilter (6) : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) @@ -153,7 +153,7 @@ : : : : : : : : : : : : : : : : : +- * Filter (108) : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (113) : : : : : : : : : : : : : : : : +- CometProject (112) : : : : : : : : : : : : : : : : +- CometFilter (111) : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) @@ -215,7 +215,7 @@ Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) CometColumnarToRow +(8) CometNativeColumnarToRow Input [2]: [sr_item_sk#14, sr_ticket_number#15] (9) BroadcastHashJoin [codegen id : 2] @@ -694,7 +694,7 @@ Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] -(113) CometColumnarToRow +(113) CometNativeColumnarToRow Input [2]: [sr_item_sk#111, sr_ticket_number#112] (114) BroadcastHashJoin [codegen id : 4] @@ -971,14 +971,14 @@ Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC 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: [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 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST] -(180) CometColumnarToRow [codegen id : 5] +(180) CometNativeColumnarToRow 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometNativeScan parquet spark_catalog.default.date_dim (181) @@ -994,7 +994,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (184) BroadcastExchange @@ -1003,7 +1003,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (188) -+- * CometColumnarToRow (187) ++- CometNativeColumnarToRow (187) +- CometFilter (186) +- CometNativeScan parquet spark_catalog.default.date_dim (185) @@ -1019,7 +1019,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(187) CometColumnarToRow [codegen id : 1] +(187) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (188) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt index 6e3f37148b..98db64e5b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -49,10 +49,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -171,10 +171,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/simplified.txt index f954616bb5..84648a9f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/simplified.txt @@ -1,206 +1,200 @@ -WholeStageCodegen (5) - 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] - 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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_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] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +CometNativeColumnarToRow + 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] + 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + WholeStageCodegen (2) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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_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] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometNativeScan 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + WholeStageCodegen (4) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #21 + WholeStageCodegen (3) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [d_date_sk,d_year] #12 ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/explain.txt index cbe790cd00..42668314fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/explain.txt @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/simplified.txt index 43527978d8..bfac9da68d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/simplified.txt @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt index cbe790cd00..42668314fd 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 @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject 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 43527978d8..bfac9da68d 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 @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/explain.txt index 7cbfb795e8..10f61d3b97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/explain.txt @@ -6,13 +6,13 @@ TakeOrderedAndProject (43) : +- * BroadcastHashJoin Inner BuildRight (23) : :- * Project (17) : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * CometColumnarToRow (3) + : : :- CometNativeColumnarToRow (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.store (1) : : +- BroadcastExchange (15) : : +- * Filter (14) : : +- * HashAggregate (13) - : : +- * CometColumnarToRow (12) + : : +- CometNativeColumnarToRow (12) : : +- CometColumnarExchange (11) : : +- * HashAggregate (10) : : +- * Project (9) @@ -22,18 +22,18 @@ TakeOrderedAndProject (43) : : : +- Scan parquet spark_catalog.default.store_sales (4) : : +- ReusedExchange (7) : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometProject (20) : +- CometFilter (19) : +- CometNativeScan parquet spark_catalog.default.item (18) +- BroadcastExchange (40) +- * Filter (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * HashAggregate (34) - +- * CometColumnarToRow (33) + +- CometNativeColumnarToRow (33) +- CometColumnarExchange (32) +- * HashAggregate (31) +- * Project (30) @@ -55,7 +55,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) CometColumnarToRow [codegen id : 9] +(3) CometNativeColumnarToRow Input [2]: [s_store_sk#1, s_store_name#2] (4) Scan parquet spark_catalog.default.store_sales @@ -66,27 +66,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -(6) Filter [codegen id : 2] +(6) Filter [codegen id : 1] Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) (7) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#8] -(8) BroadcastHashJoin [codegen id : 2] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join type: Inner Join condition: None -(9) Project [codegen id : 2] +(9) Project [codegen id : 1] Output [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] -(10) HashAggregate [codegen id : 2] +(10) HashAggregate [codegen id : 1] Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] @@ -97,17 +97,17 @@ Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] -(14) Filter [codegen id : 3] +(14) Filter [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Condition : isnotnull(revenue#12) @@ -115,13 +115,13 @@ Condition : isnotnull(revenue#12) Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 9] +(16) BroadcastHashJoin [codegen id : 6] Left keys [1]: [s_store_sk#1] Right keys [1]: [ss_store_sk#4] Join type: Inner Join condition: None -(17) Project [codegen id : 9] +(17) Project [codegen id : 6] Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] @@ -140,20 +140,20 @@ Condition : isnotnull(i_item_sk#13) Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Arguments: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18], [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#17, 50, true, false, true) AS i_brand#18] -(21) CometColumnarToRow [codegen id : 4] +(21) CometNativeColumnarToRow Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] (22) BroadcastExchange Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_item_sk#3] Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(24) Project [codegen id : 9] +(24) Project [codegen id : 6] Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] @@ -165,27 +165,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 6] +(26) ColumnarToRow [codegen id : 3] Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -(27) Filter [codegen id : 6] +(27) Filter [codegen id : 3] Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] Condition : isnotnull(ss_store_sk#20) (28) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#23] -(29) BroadcastHashJoin [codegen id : 6] +(29) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(30) Project [codegen id : 6] +(30) Project [codegen id : 3] Output [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] Input [5]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#23] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 3] Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] Keys [2]: [ss_store_sk#20, ss_item_sk#19] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] @@ -196,17 +196,17 @@ Results [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(33) CometColumnarToRow [codegen id : 7] +(33) CometNativeColumnarToRow Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -(34) HashAggregate [codegen id : 7] +(34) HashAggregate [codegen id : 4] Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] Keys [2]: [ss_store_sk#20, ss_item_sk#19] Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#21))#26] Results [2]: [ss_store_sk#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#21))#26,17,2) AS revenue#27] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 4] Input [2]: [ss_store_sk#20, revenue#27] Keys [1]: [ss_store_sk#20] Functions [1]: [partial_avg(revenue#27)] @@ -217,17 +217,17 @@ Results [3]: [ss_store_sk#20, sum#30, count#31] Input [3]: [ss_store_sk#20, sum#30, count#31] Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [3]: [ss_store_sk#20, sum#30, count#31] -(38) HashAggregate [codegen id : 8] +(38) HashAggregate [codegen id : 5] Input [3]: [ss_store_sk#20, sum#30, count#31] Keys [1]: [ss_store_sk#20] Functions [1]: [avg(revenue#27)] Aggregate Attributes [1]: [avg(revenue#27)#32] Results [2]: [ss_store_sk#20, avg(revenue#27)#32 AS ave#33] -(39) Filter [codegen id : 8] +(39) Filter [codegen id : 5] Input [2]: [ss_store_sk#20, ave#33] Condition : isnotnull(ave#33) @@ -235,13 +235,13 @@ Condition : isnotnull(ave#33) Input [2]: [ss_store_sk#20, ave#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#20] Join type: Inner Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) -(42) Project [codegen id : 9] +(42) Project [codegen id : 6] Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18, ss_store_sk#20, ave#33] @@ -253,7 +253,7 @@ Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -274,7 +274,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_mo Input [2]: [d_date_sk#8, d_month_seq#34] Arguments: [d_date_sk#8], [d_date_sk#8] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt index b1bae81b20..73e3aa7046 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt @@ -5,13 +5,13 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,28 +21,28 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -52,7 +52,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/simplified.txt index 2695c9fb89..98833264f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) + WholeStageCodegen (6) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [s_store_name,ss_store_sk,ss_item_sk,revenue] BroadcastHashJoin [s_store_sk,ss_store_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_store_name] CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [revenue] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] Project [ss_item_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -28,37 +28,33 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometNativeColumnarToRow + CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #5 - WholeStageCodegen (8) + WholeStageCodegen (5) Filter [ave] HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_store_sk] #6 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_store_sk,ss_item_sk] #7 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] Project [ss_item_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/explain.txt index 8b38832289..98d7e173d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/explain.txt @@ -231,7 +231,7 @@ Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt index 1fc09bd2e3..755be00004 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/simplified.txt index cd1c84b6f2..bfc2fc8751 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt index 8b38832289..98d7e173d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt @@ -231,7 +231,7 @@ Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/extended.txt index 1fc09bd2e3..755be00004 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 cd1c84b6f2..bfc2fc8751 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/explain.txt index 56c5025634..33fb160f49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (56) +- * HashAggregate (55) - +- * CometColumnarToRow (54) + +- CometNativeColumnarToRow (54) +- CometColumnarExchange (53) +- * HashAggregate (52) +- Union (51) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * Project (27) @@ -21,23 +21,23 @@ TakeOrderedAndProject (56) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.web_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) : : : +- ReusedExchange (11) : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) + : : +- CometNativeColumnarToRow (17) : : +- CometProject (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.time_dim (14) : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) + : +- CometNativeColumnarToRow (24) : +- CometProject (23) : +- CometFilter (22) : +- CometNativeScan parquet spark_catalog.default.ship_mode (21) +- * HashAggregate (50) - +- * CometColumnarToRow (49) + +- CometNativeColumnarToRow (49) +- CometColumnarExchange (48) +- * HashAggregate (47) +- * Project (46) @@ -65,10 +65,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) @@ -87,33 +87,33 @@ Condition : isnotnull(w_warehouse_sk#9) Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#14, 2, true, false, true) AS w_state#16, w_country#15] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] (8) BroadcastExchange Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_warehouse_sk#3] Right keys [1]: [w_warehouse_sk#9] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] (11) ReusedExchange [Reuses operator id: 60] Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#7] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] @@ -132,20 +132,20 @@ Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= Input [2]: [t_time_sk#20, t_time#21] Arguments: [t_time_sk#20], [t_time_sk#20] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [t_time_sk#20] (18) BroadcastExchange Input [1]: [t_time_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_time_sk#1] Right keys [1]: [t_time_sk#20] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] @@ -164,24 +164,24 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [1]: [sm_ship_mode_sk#22] (25) BroadcastExchange Input [1]: [sm_ship_mode_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_ship_mode_sk#2] Right keys [1]: [sm_ship_mode_sk#22] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] @@ -192,10 +192,10 @@ Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] @@ -210,66 +210,66 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#176), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 11] +(33) ColumnarToRow [codegen id : 3] Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] -(34) Filter [codegen id : 11] +(34) Filter [codegen id : 3] Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] Condition : ((isnotnull(cs_warehouse_sk#172) AND isnotnull(cs_sold_time_sk#170)) AND isnotnull(cs_ship_mode_sk#171)) (35) ReusedExchange [Reuses operator id: 8] Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -(36) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_warehouse_sk#172] Right keys [1]: [w_warehouse_sk#177] Join type: Inner Join condition: None -(37) Project [codegen id : 11] +(37) Project [codegen id : 3] Output [12]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] (38) ReusedExchange [Reuses operator id: 60] Output [3]: [d_date_sk#184, d_year#185, d_moy#186] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#176] Right keys [1]: [d_date_sk#184] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 3] Output [13]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [15]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] (41) ReusedExchange [Reuses operator id: 18] Output [1]: [t_time_sk#187] -(42) BroadcastHashJoin [codegen id : 11] +(42) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_time_sk#170] Right keys [1]: [t_time_sk#187] Join type: Inner Join condition: None -(43) Project [codegen id : 11] +(43) Project [codegen id : 3] Output [12]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] (44) ReusedExchange [Reuses operator id: 25] Output [1]: [sm_ship_mode_sk#188] -(45) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_ship_mode_sk#171] Right keys [1]: [sm_ship_mode_sk#188] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(46) Project [codegen id : 3] Output [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [13]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] -(47) HashAggregate [codegen id : 11] +(47) HashAggregate [codegen id : 3] Input [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] @@ -280,10 +280,10 @@ Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(49) CometColumnarToRow [codegen id : 12] +(49) CometNativeColumnarToRow Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -(50) HashAggregate [codegen id : 12] +(50) HashAggregate [codegen id : 4] Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] @@ -292,7 +292,7 @@ Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county (51) Union -(52) HashAggregate [codegen id : 13] +(52) HashAggregate [codegen id : 5] Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#146, feb_sales#147, mar_sales#148, apr_sales#149, may_sales#150, jun_sales#151, jul_sales#152, aug_sales#153, sep_sales#154, oct_sales#155, nov_sales#156, dec_sales#157, jan_net#158, feb_net#159, mar_net#160, apr_net#161, may_net#162, jun_net#163, jul_net#164, aug_net#165, sep_net#166, oct_net#167, nov_net#168, dec_net#169] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] Functions [36]: [partial_sum(jan_sales#146), partial_sum(feb_sales#147), partial_sum(mar_sales#148), partial_sum(apr_sales#149), partial_sum(may_sales#150), partial_sum(jun_sales#151), partial_sum(jul_sales#152), partial_sum(aug_sales#153), partial_sum(sep_sales#154), partial_sum(oct_sales#155), partial_sum(nov_sales#156), partial_sum(dec_sales#157), partial_sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#158), partial_sum(feb_net#159), partial_sum(mar_net#160), partial_sum(apr_net#161), partial_sum(may_net#162), partial_sum(jun_net#163), partial_sum(jul_net#164), partial_sum(aug_net#165), partial_sum(sep_net#166), partial_sum(oct_net#167), partial_sum(nov_net#168), partial_sum(dec_net#169)] @@ -303,10 +303,10 @@ Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(54) CometColumnarToRow [codegen id : 14] +(54) CometNativeColumnarToRow Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -(55) HashAggregate [codegen id : 14] +(55) HashAggregate [codegen id : 6] Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] Functions [36]: [sum(jan_sales#146), sum(feb_sales#147), sum(mar_sales#148), sum(apr_sales#149), sum(may_sales#150), sum(jun_sales#151), sum(jul_sales#152), sum(aug_sales#153), sum(sep_sales#154), sum(oct_sales#155), sum(nov_sales#156), sum(dec_sales#157), sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#158), sum(feb_net#159), sum(mar_net#160), sum(apr_net#161), sum(may_net#162), sum(jun_net#163), sum(jul_net#164), sum(aug_net#165), sum(sep_net#166), sum(oct_net#167), sum(nov_net#168), sum(dec_net#169)] @@ -321,7 +321,7 @@ Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_w Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (60) -+- * CometColumnarToRow (59) ++- CometNativeColumnarToRow (59) +- CometFilter (58) +- CometNativeScan parquet spark_catalog.default.date_dim (57) @@ -337,7 +337,7 @@ ReadSchema: struct Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) -(59) CometColumnarToRow [codegen id : 1] +(59) CometNativeColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_moy#19] (60) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt index caf49b6d52..3f3c6b8d79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,30 +21,30 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.time_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.ship_mode +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -60,21 +60,21 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.time_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/simplified.txt index 3a1f053d60..7ceb1fb345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) + WholeStageCodegen (6) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) + WholeStageCodegen (5) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] @@ -29,43 +29,35 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometNativeColumnarToRow + CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] InputAdapter ReusedExchange [d_date_sk,d_year,d_moy] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + CometNativeColumnarToRow + CometProject [t_time_sk] + CometFilter [t_time_sk,t_time] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) + CometNativeColumnarToRow + CometProject [sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] + CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) + WholeStageCodegen (3) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/explain.txt index 7599e9ecd3..93b64a4b6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/explain.txt @@ -289,7 +289,7 @@ Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -305,7 +305,7 @@ ReadSchema: struct Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_moy#19] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt index ca39e001b0..7e7f77c2c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/simplified.txt index bc6bde24b2..f3122945a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/simplified.txt @@ -21,11 +21,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt index 7599e9ecd3..93b64a4b6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt @@ -289,7 +289,7 @@ Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -305,7 +305,7 @@ ReadSchema: struct Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_moy#19] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/extended.txt index ca39e001b0..7e7f77c2c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange 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 bc6bde24b2..f3122945a6 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 @@ -21,11 +21,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/explain.txt index 547ca84576..ccfe27f8b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/explain.txt @@ -3,13 +3,13 @@ TakeOrderedAndProject (34) +- * Filter (33) +- Window (32) +- WindowGroupLimit (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometSort (29) +- CometColumnarExchange (28) +- WindowGroupLimit (27) +- * Sort (26) +- * HashAggregate (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometColumnarExchange (23) +- * HashAggregate (22) +- * Expand (21) @@ -24,12 +24,12 @@ TakeOrderedAndProject (34) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.store (7) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometProject (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.item (14) @@ -43,23 +43,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 39] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] @@ -78,20 +78,20 @@ Condition : isnotnull(s_store_sk#11) Input [2]: [s_store_sk#11, s_store_id#12] Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [s_store_sk#11, s_store_id#13] (11) BroadcastExchange Input [2]: [s_store_sk#11, s_store_id#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] @@ -110,28 +110,28 @@ Condition : isnotnull(i_item_sk#14) Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] (18) BroadcastExchange Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(21) Expand [codegen id : 4] +(21) Expand [codegen id : 1] Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#21, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 1] Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -142,17 +142,17 @@ Results [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year# Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] Arguments: hashpartitioning(i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 5] +(24) CometNativeColumnarToRow Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 2] Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36] Results [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36 AS sumsales#37] -(26) Sort [codegen id : 5] +(26) Sort [codegen id : 2] Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] Arguments: [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST], false, 0 @@ -168,7 +168,7 @@ Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, CometColumna Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] Arguments: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37], [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] (31) WindowGroupLimit @@ -179,7 +179,7 @@ Arguments: [i_category#23], [sumsales#37 DESC NULLS LAST], rank(sumsales#37), 10 Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] Arguments: [rank(sumsales#37) windowspecdefinition(i_category#23, sumsales#37 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [i_category#23], [sumsales#37 DESC NULLS LAST] -(33) Filter [codegen id : 7] +(33) Filter [codegen id : 3] Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] Condition : (rk#38 <= 100) @@ -191,7 +191,7 @@ Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometNativeScan parquet spark_catalog.default.date_dim (35) @@ -212,7 +212,7 @@ Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1200)) AND (d_mo Input [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt index e64576b0f0..da753c17c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt @@ -2,13 +2,13 @@ TakeOrderedAndProject +- Filter +- Window +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- WindowGroupLimit +- Sort +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -23,22 +23,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/simplified.txt index 7c5b24a903..31f28feccf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/simplified.txt @@ -1,57 +1,49 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) + WholeStageCodegen (3) Filter [rk] InputAdapter Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (5) - Sort [i_category,sumsales] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (2) + Sort [i_category,sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/explain.txt index 3e9bbb06ae..0bf12f486e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -212,7 +212,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt index 7504243c7a..d03ce0e111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/simplified.txt index 9069117a5b..b4f46e52ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/simplified.txt @@ -28,12 +28,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt index 3e9bbb06ae..0bf12f486e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt @@ -191,7 +191,7 @@ Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -212,7 +212,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/extended.txt index 7504243c7a..d03ce0e111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 9069117a5b..b4f46e52ae 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 @@ -28,12 +28,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/explain.txt index 7da946625e..e9ff6848dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (41) :- * Project (37) : +- * BroadcastHashJoin Inner BuildRight (36) : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) + : : +- CometNativeColumnarToRow (29) : : +- CometColumnarExchange (28) : : +- * HashAggregate (27) : : +- * Project (26) @@ -21,21 +21,21 @@ TakeOrderedAndProject (41) : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : +- ReusedExchange (4) : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) + : : : : +- CometNativeColumnarToRow (10) : : : : +- CometProject (9) : : : : +- CometFilter (8) : : : : +- CometNativeScan parquet spark_catalog.default.store (7) : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) + : : : +- CometNativeColumnarToRow (17) : : : +- CometProject (16) : : : +- CometFilter (15) : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometFilter (22) : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) + : +- CometNativeColumnarToRow (34) : +- CometProject (33) : +- CometFilter (32) : +- CometNativeScan parquet spark_catalog.default.customer (31) @@ -50,23 +50,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#11] -(5) BroadcastHashJoin [codegen id : 5] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(6) Project [codegen id : 5] +(6) Project [codegen id : 1] Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] @@ -85,20 +85,20 @@ Condition : (s_city#13 IN (Midway,Fairview) AND isnotnull(s_store_sk#12)) Input [2]: [s_store_sk#12, s_city#13] Arguments: [s_store_sk#12], [s_store_sk#12] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#12] (11) BroadcastExchange Input [1]: [s_store_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#12] @@ -117,20 +117,20 @@ Condition : (((hd_dep_count#15 = 4) OR (hd_vehicle_count#16 = 3)) AND isnotnull( Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] Arguments: [hd_demo_sk#14], [hd_demo_sk#14] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#14] (18) BroadcastExchange Input [1]: [hd_demo_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#14] @@ -145,24 +145,24 @@ ReadSchema: struct Input [2]: [ca_address_sk#17, ca_city#18] Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_city#18)) -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [ca_address_sk#17, ca_city#18] (24) BroadcastExchange Input [2]: [ca_address_sk#17, ca_city#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#3] Right keys [1]: [ca_address_sk#17] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(26) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#17, ca_city#18] -(27) HashAggregate [codegen id : 5] +(27) HashAggregate [codegen id : 1] Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] @@ -173,10 +173,10 @@ Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, su Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometColumnarToRow [codegen id : 8] +(29) CometNativeColumnarToRow Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -(30) HashAggregate [codegen id : 8] +(30) HashAggregate [codegen id : 2] Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] @@ -198,33 +198,33 @@ Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] Arguments: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37], [c_customer_sk#32, c_current_addr_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#37] -(34) CometColumnarToRow [codegen id : 6] +(34) CometNativeColumnarToRow Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] (35) BroadcastExchange Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 2] Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37] Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] (38) ReusedExchange [Reuses operator id: 24] Output [2]: [ca_address_sk#38, ca_city#39] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_current_addr_sk#33] Right keys [1]: [ca_address_sk#38] Join type: Inner Join condition: NOT (ca_city#39 = bought_city#28) -(40) Project [codegen id : 8] +(40) Project [codegen id : 2] Output [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_city#39] @@ -236,7 +236,7 @@ Arguments: 100, [c_last_name#37 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FI Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometNativeScan parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((((isnotnull(d_dom#41) AND (d_dom#41 >= 1)) AND (d_dom#41 <= 2)) AN Input [3]: [d_date_sk#11, d_year#40, d_dom#41] Arguments: [d_date_sk#11], [d_date_sk#11] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt index c122bf3803..cebcd85144 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -20,36 +20,36 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/simplified.txt index 05b883b1e6..5a807ddd72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -24,44 +24,34 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometNativeColumnarToRow + CometFilter [ca_address_sk,ca_city] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/explain.txt index bc17101a84..23a09a8dd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/explain.txt @@ -225,7 +225,7 @@ Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/simplified.txt index fdddd82de0..ee6be5a221 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt index bc17101a84..23a09a8dd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt @@ -225,7 +225,7 @@ Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fdddd82de0..ee6be5a221 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/explain.txt index 556dd015a5..c2cdd107b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (44) +- * HashAggregate (43) - +- * CometColumnarToRow (42) + +- CometNativeColumnarToRow (42) +- CometColumnarExchange (41) +- * HashAggregate (40) +- * Project (39) @@ -12,7 +12,7 @@ TakeOrderedAndProject (44) : : +- * BroadcastHashJoin LeftAnti BuildRight (24) : : :- * BroadcastHashJoin LeftAnti BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -34,12 +34,12 @@ TakeOrderedAndProject (44) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (30) - : +- * CometColumnarToRow (29) + : +- CometNativeColumnarToRow (29) : +- CometProject (28) : +- CometFilter (27) : +- CometNativeScan parquet spark_catalog.default.customer_address (26) +- BroadcastExchange (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.customer_demographics (33) @@ -56,7 +56,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] (6) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] @@ -86,7 +86,7 @@ Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi @@ -99,19 +99,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] (13) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#10] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#8] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] @@ -119,7 +119,7 @@ Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] Input [1]: [ws_bill_customer_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#8] Join type: LeftAnti @@ -132,19 +132,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#6)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] (20) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#13] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#11] Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] @@ -152,13 +152,13 @@ Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] Input [1]: [cs_ship_customer_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#11] Join type: LeftAnti Join condition: None -(25) Project [codegen id : 9] +(25) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -177,20 +177,20 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [ca_address_sk#14, ca_state#15] Arguments: [ca_address_sk#14], [ca_address_sk#14] -(29) CometColumnarToRow [codegen id : 7] +(29) CometNativeColumnarToRow Input [1]: [ca_address_sk#14] (30) BroadcastExchange Input [1]: [ca_address_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 9] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#14] Join type: Inner Join condition: None -(32) Project [codegen id : 9] +(32) Project [codegen id : 4] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#14] @@ -209,24 +209,24 @@ Condition : isnotnull(cd_demo_sk#16) Input [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] Arguments: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25], [cd_demo_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#17, 1, true, false, true) AS cd_gender#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#18, 1, true, false, true) AS cd_marital_status#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#19, 20, true, false, true) AS cd_education_status#24, cd_purchase_estimate#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#21, 10, true, false, true) AS cd_credit_rating#25] -(36) CometColumnarToRow [codegen id : 8] +(36) CometNativeColumnarToRow Input [6]: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] (37) BroadcastExchange Input [6]: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#16] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(39) Project [codegen id : 4] Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 4] Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Functions [1]: [partial_count(1)] @@ -237,10 +237,10 @@ Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_pur Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometColumnarToRow [codegen id : 10] +(42) CometNativeColumnarToRow Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] -(43) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 5] Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Functions [1]: [count(1)] @@ -255,7 +255,7 @@ Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FI Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometNativeScan parquet spark_catalog.default.date_dim (45) @@ -276,7 +276,7 @@ Condition : (((((isnotnull(d_year#32) AND isnotnull(d_moy#33)) AND (d_year#32 = Input [3]: [d_date_sk#7, d_year#32, d_moy#33] Arguments: [d_date_sk#7], [d_date_sk#7] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt index 821b101f54..f48c67d6e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -21,12 +21,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,7 +37,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -48,17 +48,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/simplified.txt index f8868e38be..e5b4d44c3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -14,13 +14,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -28,17 +28,15 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -49,7 +47,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -60,17 +58,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/explain.txt index af81415f33..341ddd9ffc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/explain.txt @@ -36,12 +36,12 @@ TakeOrderedAndProject (46) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) + : +- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometFilter (29) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) +- BroadcastExchange (39) - +- * CometColumnarToRow (38) + +- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) @@ -102,7 +102,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -131,7 +131,7 @@ Input [1]: [ws_bill_customer_sk#10] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#10] Join type: LeftAnti @@ -163,13 +163,13 @@ Input [1]: [cs_ship_customer_sk#14] Input [1]: [cs_ship_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#14] Join type: LeftAnti Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -188,20 +188,20 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [ca_address_sk#18] (32) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 5] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(34) Project [codegen id : 5] +(34) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] @@ -220,24 +220,24 @@ Condition : isnotnull(cd_demo_sk#20) Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Arguments: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#28, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#29] -(38) CometColumnarToRow [codegen id : 4] +(38) CometNativeColumnarToRow Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] (39) BroadcastExchange Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 5] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(41) Project [codegen id : 5] +(41) Project [codegen id : 3] Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -(42) HashAggregate [codegen id : 5] +(42) HashAggregate [codegen id : 3] Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Functions [1]: [partial_count(1)] @@ -248,10 +248,10 @@ Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_pur Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) CometColumnarToRow [codegen id : 6] +(44) CometColumnarToRow [codegen id : 4] Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -(45) HashAggregate [codegen id : 6] +(45) HashAggregate [codegen id : 4] Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Functions [1]: [count(1)] @@ -266,7 +266,7 @@ Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (51) -+- * CometColumnarToRow (50) ++- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) @@ -287,7 +287,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(50) CometColumnarToRow [codegen id : 1] +(50) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt index b4fbb455ef..5a472217df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -49,12 +49,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/simplified.txt index 68bf32c40b..fd8db7bed2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -24,12 +24,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -56,17 +54,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt index af81415f33..341ddd9ffc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt @@ -36,12 +36,12 @@ TakeOrderedAndProject (46) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) + : +- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometFilter (29) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) +- BroadcastExchange (39) - +- * CometColumnarToRow (38) + +- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) @@ -102,7 +102,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -131,7 +131,7 @@ Input [1]: [ws_bill_customer_sk#10] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#10] Join type: LeftAnti @@ -163,13 +163,13 @@ Input [1]: [cs_ship_customer_sk#14] Input [1]: [cs_ship_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#14] Join type: LeftAnti Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -188,20 +188,20 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [ca_address_sk#18] (32) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 5] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(34) Project [codegen id : 5] +(34) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] @@ -220,24 +220,24 @@ Condition : isnotnull(cd_demo_sk#20) Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Arguments: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#28, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#29] -(38) CometColumnarToRow [codegen id : 4] +(38) CometNativeColumnarToRow Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] (39) BroadcastExchange Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 5] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(41) Project [codegen id : 5] +(41) Project [codegen id : 3] Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -(42) HashAggregate [codegen id : 5] +(42) HashAggregate [codegen id : 3] Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Functions [1]: [partial_count(1)] @@ -248,10 +248,10 @@ Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_pur Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) CometColumnarToRow [codegen id : 6] +(44) CometColumnarToRow [codegen id : 4] Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -(45) HashAggregate [codegen id : 6] +(45) HashAggregate [codegen id : 4] Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Functions [1]: [count(1)] @@ -266,7 +266,7 @@ Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (51) -+- * CometColumnarToRow (50) ++- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) @@ -287,7 +287,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(50) CometColumnarToRow [codegen id : 1] +(50) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/extended.txt index b4fbb455ef..5a472217df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -49,12 +49,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt index 68bf32c40b..fd8db7bed2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -24,12 +24,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -56,17 +54,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/explain.txt index 1a5a7efc91..b177b3ca39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * HashAggregate (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * Project (27) @@ -16,18 +16,18 @@ TakeOrderedAndProject (32) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : +- ReusedExchange (11) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.item (14) +- BroadcastExchange (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometNativeScan parquet spark_catalog.default.promotion (21) @@ -41,10 +41,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) @@ -63,33 +63,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -108,20 +108,20 @@ Condition : isnotnull(i_item_sk#15) Input [2]: [i_item_sk#15, i_item_id#16] Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#16, 16, true, false, true) AS i_item_id#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [i_item_sk#15, i_item_id#17] (18) BroadcastExchange Input [2]: [i_item_sk#15, i_item_id#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#15, i_item_id#17] @@ -140,24 +140,24 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Arguments: [p_promo_sk#18], [p_promo_sk#18] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [1]: [p_promo_sk#18] (25) BroadcastExchange Input [1]: [p_promo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_promo_sk#3] Right keys [1]: [p_promo_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17, p_promo_sk#18] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] @@ -168,10 +168,10 @@ Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34 Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Keys [1]: [i_item_id#17] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] @@ -186,7 +186,7 @@ Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#45] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt index f40fdbe4e3..b692dfa9be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -16,27 +16,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/simplified.txt index bb670b4a73..d7988608ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] BroadcastHashJoin [ss_promo_sk,p_promo_sk] @@ -20,35 +20,27 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometNativeColumnarToRow + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/explain.txt index 0e32283618..01acebb66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt index 655f651a41..3aa5ad7551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/simplified.txt index a15bcd5c0d..707bfe627e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/simplified.txt @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt index 0e32283618..01acebb66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/extended.txt index 655f651a41..3aa5ad7551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a15bcd5c0d..707bfe627e 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 @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/explain.txt index c1925c206e..9d068aeb14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- CometColumnarExchange (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Expand (38) @@ -21,7 +21,7 @@ TakeOrderedAndProject (48) +- BroadcastExchange (35) +- * Project (34) +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (9) + :- CometNativeColumnarToRow (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.store (7) +- BroadcastExchange (32) @@ -31,7 +31,7 @@ TakeOrderedAndProject (48) +- WindowGroupLimit (28) +- * Sort (27) +- * HashAggregate (26) - +- * CometColumnarToRow (25) + +- CometNativeColumnarToRow (25) +- CometColumnarExchange (24) +- * HashAggregate (23) +- * Project (22) @@ -42,7 +42,7 @@ TakeOrderedAndProject (48) : : +- * ColumnarToRow (11) : : +- Scan parquet spark_catalog.default.store_sales (10) : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometProject (15) : +- CometFilter (14) : +- CometNativeScan parquet spark_catalog.default.store (13) @@ -57,23 +57,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 5] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -88,7 +88,7 @@ ReadSchema: struct Input [3]: [s_store_sk#6, s_county#7, s_state#8] Condition : isnotnull(s_store_sk#6) -(9) CometColumnarToRow [codegen id : 7] +(9) CometNativeColumnarToRow Input [3]: [s_store_sk#6, s_county#7, s_state#8] (10) Scan parquet spark_catalog.default.store_sales @@ -99,10 +99,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Condition : isnotnull(ss_store_sk#9) @@ -121,37 +121,37 @@ Condition : isnotnull(s_store_sk#12) Input [2]: [s_store_sk#12, s_state#13] Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#14] -(16) CometColumnarToRow [codegen id : 2] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#12, s_state#14] (17) BroadcastExchange Input [2]: [s_store_sk#12, s_state#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#9] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] (20) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 4] +(21) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#11] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 4] +(22) Project [codegen id : 1] Output [2]: [ss_net_profit#10, s_state#14] Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] -(23) HashAggregate [codegen id : 4] +(23) HashAggregate [codegen id : 1] Input [2]: [ss_net_profit#10, s_state#14] Keys [1]: [s_state#14] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] @@ -162,17 +162,17 @@ Results [2]: [s_state#14, sum#17] Input [2]: [s_state#14, sum#17] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(25) CometColumnarToRow [codegen id : 5] +(25) CometNativeColumnarToRow Input [2]: [s_state#14, sum#17] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 2] Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(27) Sort [codegen id : 5] +(27) Sort [codegen id : 2] Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 @@ -184,11 +184,11 @@ Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] -(30) Filter [codegen id : 6] +(30) Filter [codegen id : 3] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Condition : (ranking#20 <= 5) -(31) Project [codegen id : 6] +(31) Project [codegen id : 3] Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] @@ -196,13 +196,13 @@ Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 7] +(33) BroadcastHashJoin [codegen id : 4] Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(34) Project [codegen id : 7] +(34) Project [codegen id : 4] Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#21] Input [3]: [s_store_sk#6, s_county#7, s_state#8] @@ -210,21 +210,21 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] Input [3]: [s_store_sk#6, s_county#7, s_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 5] Output [3]: [ss_net_profit#2, s_state#21, s_county#7] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] -(38) Expand [codegen id : 8] +(38) Expand [codegen id : 5] Input [3]: [ss_net_profit#2, s_state#21, s_county#7] Arguments: [[ss_net_profit#2, s_state#21, s_county#7, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 5] Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] @@ -235,10 +235,10 @@ Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 6] Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] @@ -253,14 +253,14 @@ Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] -(45) CometColumnarToRow [codegen id : 10] +(45) CometNativeColumnarToRow Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] (46) Window Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] -(47) Project [codegen id : 11] +(47) Project [codegen id : 7] Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] @@ -272,7 +272,7 @@ Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -293,7 +293,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#34] Arguments: [d_date_sk#5], [d_date_sk#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt index 4832534e6e..f222353c17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -18,19 +18,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange @@ -40,7 +40,7 @@ TakeOrderedAndProject +- WindowGroupLimit +- Sort +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -52,12 +52,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/simplified.txt index f61238590a..21c9c4dbc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/simplified.txt @@ -1,80 +1,74 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) + WholeStageCodegen (7) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (9) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (6) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + InputAdapter + CometNativeColumnarToRow + CometFilter [s_store_sk,s_county,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (2) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state] #6 + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/explain.txt index 95dc8874aa..d7fac724f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/explain.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject (49) +- BroadcastExchange (36) +- * Project (35) +- * BroadcastHashJoin LeftSemi BuildRight (34) - :- * CometColumnarToRow (12) + :- CometNativeColumnarToRow (12) : +- CometFilter (11) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) +- BroadcastExchange (33) @@ -104,7 +104,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -276,7 +276,7 @@ Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -297,7 +297,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt index 45a2c7a669..0239787721 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -29,7 +29,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/simplified.txt index 80f0cfc8c2..d3262b3181 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -41,8 +39,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/explain.txt index 95dc8874aa..d7fac724f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/explain.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject (49) +- BroadcastExchange (36) +- * Project (35) +- * BroadcastHashJoin LeftSemi BuildRight (34) - :- * CometColumnarToRow (12) + :- CometNativeColumnarToRow (12) : +- CometFilter (11) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) +- BroadcastExchange (33) @@ -104,7 +104,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -276,7 +276,7 @@ Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -297,7 +297,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt index 45a2c7a669..0239787721 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -29,7 +29,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange 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 80f0cfc8c2..d3262b3181 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -41,8 +39,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/explain.txt index 7c4f959123..81be326835 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (40) +CometNativeColumnarToRow (40) +- CometSort (39) +- CometColumnarExchange (38) +- * HashAggregate (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometColumnarExchange (35) +- * HashAggregate (34) +- * Project (33) @@ -11,7 +11,7 @@ :- * Project (26) : +- * BroadcastHashJoin Inner BuildLeft (25) : :- BroadcastExchange (5) - : : +- * CometColumnarToRow (4) + : : +- CometNativeColumnarToRow (4) : : +- CometProject (3) : : +- CometFilter (2) : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -35,7 +35,7 @@ : : +- Scan parquet spark_catalog.default.store_sales (18) : +- ReusedExchange (21) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.time_dim (27) @@ -56,7 +56,7 @@ Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull( Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#3, 50, true, false, true) AS i_brand#5] -(4) CometColumnarToRow [codegen id : 1] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] (5) BroadcastExchange @@ -71,23 +71,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 3] +(7) ColumnarToRow [codegen id : 1] Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -(8) Filter [codegen id : 3] +(8) Filter [codegen id : 1] Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) (9) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#11] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 1] Output [3]: [ws_ext_sales_price#8 AS ext_price#12, ws_item_sk#7 AS sold_item_sk#13, ws_sold_time_sk#6 AS time_sk#14] Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] @@ -99,23 +99,23 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 5] +(13) ColumnarToRow [codegen id : 2] Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -(14) Filter [codegen id : 5] +(14) Filter [codegen id : 2] Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] Condition : (isnotnull(cs_item_sk#16) AND isnotnull(cs_sold_time_sk#15)) (15) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#19] -(16) BroadcastHashJoin [codegen id : 5] +(16) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(17) Project [codegen id : 5] +(17) Project [codegen id : 2] Output [3]: [cs_ext_sales_price#17 AS ext_price#20, cs_item_sk#16 AS sold_item_sk#21, cs_sold_time_sk#15 AS time_sk#22] Input [5]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#19] @@ -127,35 +127,35 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 3] Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(20) Filter [codegen id : 7] +(20) Filter [codegen id : 3] Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) (21) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#27] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 3] Output [3]: [ss_ext_sales_price#25 AS ext_price#28, ss_item_sk#24 AS sold_item_sk#29, ss_sold_time_sk#23 AS time_sk#30] Input [5]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] (24) Union -(25) BroadcastHashJoin [codegen id : 9] +(25) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] Right keys [1]: [sold_item_sk#13] Join type: Inner Join condition: None -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [4]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#12, sold_item_sk#13, time_sk#14] @@ -174,24 +174,24 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] Arguments: [t_time_sk#31, t_hour#32, t_minute#33], [t_time_sk#31, t_hour#32, t_minute#33] -(30) CometColumnarToRow [codegen id : 8] +(30) CometNativeColumnarToRow Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] (31) BroadcastExchange Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [time_sk#14] Right keys [1]: [t_time_sk#31] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] Input [7]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14, t_time_sk#31, t_hour#32, t_minute#33] -(34) HashAggregate [codegen id : 9] +(34) HashAggregate [codegen id : 4] Input [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] Functions [1]: [partial_sum(UnscaledValue(ext_price#12))] @@ -202,10 +202,10 @@ Results [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(36) CometColumnarToRow [codegen id : 10] +(36) CometNativeColumnarToRow Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] -(37) HashAggregate [codegen id : 10] +(37) HashAggregate [codegen id : 5] Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] Functions [1]: [sum(UnscaledValue(ext_price#12))] @@ -220,14 +220,14 @@ Arguments: rangepartitioning(ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] Arguments: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40], [ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 11] +(40) CometNativeColumnarToRow Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.date_dim (41) @@ -248,7 +248,7 @@ Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 11 Input [3]: [d_date_sk#11, d_year#41, d_moy#42] Arguments: [d_date_sk#11], [d_date_sk#11] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt index 427d3518dc..23844f41f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ CometColumnarToRow :- Project : +- BroadcastHashJoin : :- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -22,12 +22,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,12 +49,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/simplified.txt index 838a3e6604..9fdaac52c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/simplified.txt @@ -1,71 +1,63 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 - WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + CometSort [brand_id,brand,t_hour,t_minute,ext_price] + CometColumnarExchange [ext_price,brand_id] #1 + WholeStageCodegen (5) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + WholeStageCodegen (4) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] + Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] + BroadcastHashJoin [time_sk,t_time_sk] + Project [i_brand_id,i_brand,ext_price,time_sk] + BroadcastHashJoin [i_item_sk,sold_item_sk] + InputAdapter + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + InputAdapter + Union + WholeStageCodegen (1) + Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_time_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (2) + Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_time_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (3) + Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_time_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + Scan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/explain.txt index bb217d6417..e18049c7aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/explain.txt @@ -216,7 +216,7 @@ Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11 Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt index 25ea15c536..12dc136fee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/simplified.txt index 63e7c353ac..0c925fadcb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt index bb217d6417..e18049c7aa 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 @@ -216,7 +216,7 @@ Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11 Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/extended.txt index 25ea15c536..12dc136fee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 63e7c353ac..0c925fadcb 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/explain.txt index ed3a4a7aea..6b720bc351 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (70) +CometNativeColumnarToRow (70) +- CometTakeOrderedAndProject (69) +- CometHashAggregate (68) +- CometExchange (67) @@ -30,38 +30,38 @@ : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) + : : : : : : : : : +- CometNativeColumnarToRow (6) : : : : : : : : : +- CometFilter (5) : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) + : : : : : : : : +- CometNativeColumnarToRow (12) : : : : : : : : +- CometFilter (11) : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometNativeColumnarToRow (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) + : : : : : : +- CometNativeColumnarToRow (25) : : : : : : +- CometProject (24) : : : : : : +- CometFilter (23) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) + : : : : : +- CometNativeColumnarToRow (32) : : : : : +- CometProject (31) : : : : : +- CometFilter (30) : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) : : : : +- ReusedExchange (36) : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) + : : : +- CometNativeColumnarToRow (41) : : : +- CometFilter (40) : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) + : : +- CometNativeColumnarToRow (47) : : +- CometFilter (46) : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometFilter (52) : +- CometNativeScan parquet spark_catalog.default.promotion (51) +- CometSort (63) @@ -79,10 +79,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 10] +(2) ColumnarToRow [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -(3) Filter [codegen id : 10] +(3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) @@ -98,20 +98,20 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] (13) BroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 10] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#11] Right keys [1]: [w_warehouse_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 10] +(15) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] @@ -154,20 +154,20 @@ ReadSchema: struct Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) CometColumnarToRow [codegen id : 3] +(18) CometNativeColumnarToRow Input [2]: [i_item_sk#16, i_item_desc#17] (19) BroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 10] +(20) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#4] Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(21) Project [codegen id : 10] +(21) Project [codegen id : 1] Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] @@ -186,20 +186,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) CometColumnarToRow [codegen id : 4] +(25) CometNativeColumnarToRow Input [1]: [cd_demo_sk#18] (26) BroadcastExchange Input [1]: [cd_demo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 10] +(27) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(28) Project [codegen id : 10] +(28) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] @@ -218,33 +218,33 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) CometColumnarToRow [codegen id : 5] +(32) CometNativeColumnarToRow Input [1]: [hd_demo_sk#20] (33) BroadcastExchange Input [1]: [hd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_hdemo_sk#3] Right keys [1]: [hd_demo_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 10] +(35) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] (36) ReusedExchange [Reuses operator id: 75] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(37) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#8] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(38) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] @@ -259,20 +259,20 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_week_seq#26] Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) -(41) CometColumnarToRow [codegen id : 7] +(41) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_week_seq#26] (42) BroadcastExchange Input [2]: [d_date_sk#25, d_week_seq#26] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 10] +(43) BroadcastHashJoin [codegen id : 1] Left keys [2]: [d_week_seq#24, inv_date_sk#13] Right keys [2]: [d_week_seq#26, d_date_sk#25] Join type: Inner Join condition: None -(44) Project [codegen id : 10] +(44) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] @@ -287,20 +287,20 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_date#28] Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) -(47) CometColumnarToRow [codegen id : 8] +(47) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_date#28] (48) BroadcastExchange Input [2]: [d_date_sk#27, d_date#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: (d_date#28 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(50) Project [codegen id : 1] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] @@ -315,20 +315,20 @@ ReadSchema: struct Input [1]: [p_promo_sk#29] Condition : isnotnull(p_promo_sk#29) -(53) CometColumnarToRow [codegen id : 9] +(53) CometNativeColumnarToRow Input [1]: [p_promo_sk#29] (54) BroadcastExchange Input [1]: [p_promo_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(55) BroadcastHashJoin [codegen id : 10] +(55) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_promo_sk#5] Right keys [1]: [p_promo_sk#29] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(56) Project [codegen id : 1] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] @@ -390,14 +390,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -(70) CometColumnarToRow [codegen id : 11] +(70) CometNativeColumnarToRow Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (75) -+- * CometColumnarToRow (74) ++- CometNativeColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -418,7 +418,7 @@ Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) CometColumnarToRow [codegen id : 1] +(74) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt index 081972705a..62c2b17b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -30,47 +30,47 @@ CometColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/simplified.txt index 025217a607..edcf2b00f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/simplified.txt @@ -1,107 +1,87 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - 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] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + 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] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + BroadcastExchange #6 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + BroadcastExchange #9 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + BroadcastExchange #10 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + CometNativeColumnarToRow + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/explain.txt index bf5bcb475b..1c2d72fad3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/simplified.txt index 678f8c67df..549469c03b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt index bf5bcb475b..1c2d72fad3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 678f8c67df..549469c03b 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 @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/explain.txt index 7fe1aa758c..fcc1a09c69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometSort (34) +- CometColumnarExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) :- * Filter (25) : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -20,17 +20,17 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.customer (26) @@ -44,23 +44,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] @@ -79,20 +79,20 @@ Condition : (s_county#9 IN (Williamson County,Franklin Parish,Bronx County,Orang Input [2]: [s_store_sk#8, s_county#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (11) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] @@ -111,24 +111,24 @@ Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apac Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#10] (18) BroadcastExchange Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] @@ -139,17 +139,17 @@ Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) @@ -168,20 +168,20 @@ Condition : isnotnull(c_customer_sk#18) Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] (30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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] @@ -193,14 +193,14 @@ Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, Co 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: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (40) -+- * CometColumnarToRow (39) ++- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometNativeScan parquet spark_catalog.default.date_dim (36) @@ -221,7 +221,7 @@ Condition : ((((isnotnull(d_dom#28) AND (d_dom#28 >= 1)) AND (d_dom#28 <= 2)) AN Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(39) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (40) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt index 9e4596c12c..40230582bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- BroadcastHashJoin :- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,27 +19,27 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/simplified.txt index 4d2a6a74ab..ef3860150b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/simplified.txt @@ -1,59 +1,49 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] +CometNativeColumnarToRow + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [cnt] #1 + WholeStageCodegen (2) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/explain.txt index 346e110371..667cf5d6a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/simplified.txt index 227dd00f2b..133182ebde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 346e110371..667cf5d6a8 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 @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 227dd00f2b..133182ebde 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/explain.txt index c019668fc8..3ce14f881d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/explain.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject (79) : :- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -25,14 +25,14 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -44,14 +44,14 @@ TakeOrderedAndProject (79) : +- BroadcastExchange (56) : +- * Filter (55) : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (47) : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) + : : :- CometNativeColumnarToRow (41) : : : +- CometProject (40) : : : +- CometFilter (39) : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -62,14 +62,14 @@ TakeOrderedAndProject (79) : +- ReusedExchange (48) +- BroadcastExchange (76) +- * HashAggregate (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometColumnarExchange (73) +- * HashAggregate (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) :- * Project (68) : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) + : :- CometNativeColumnarToRow (62) : : +- CometProject (61) : : +- CometFilter (60) : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -95,7 +95,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] (5) Scan parquet spark_catalog.default.store_sales @@ -117,30 +117,30 @@ Condition : isnotnull(ss_customer_sk#8) Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] (11) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#12, d_year#13] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] @@ -151,17 +151,17 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) @@ -180,7 +180,7 @@ Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] (23) Scan parquet spark_catalog.default.store_sales @@ -191,10 +191,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Condition : isnotnull(ss_customer_sk#23) @@ -202,30 +202,30 @@ Condition : isnotnull(ss_customer_sk#23) Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#19] Right keys [1]: [ss_customer_sk#23] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#27, d_year#28] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#25] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] @@ -236,10 +236,10 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] @@ -250,7 +250,7 @@ Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_firs Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#31] Join type: Inner @@ -271,7 +271,7 @@ Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] (42) Scan parquet spark_catalog.default.web_sales @@ -282,10 +282,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Condition : isnotnull(ws_bill_customer_sk#42) @@ -293,30 +293,30 @@ Condition : isnotnull(ws_bill_customer_sk#42) Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#35] Right keys [1]: [ws_bill_customer_sk#42] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#45, d_year#46] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#44] Right keys [1]: [d_date_sk#45] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] @@ -327,17 +327,17 @@ Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum# Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#49] Results [2]: [c_customer_id#39 AS customer_id#50, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#49,17,2) AS year_total#51] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#50, year_total#51] Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) @@ -345,13 +345,13 @@ Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) Input [2]: [customer_id#50, year_total#51] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#50] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(58) Project [codegen id : 12] Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51] Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#50, year_total#51] @@ -370,7 +370,7 @@ Condition : (isnotnull(c_customer_sk#52) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] Arguments: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#54, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#55, 30, true, false, true) AS c_last_name#41] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [4]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41] (63) Scan parquet spark_catalog.default.web_sales @@ -381,10 +381,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] Condition : isnotnull(ws_bill_customer_sk#56) @@ -392,30 +392,30 @@ Condition : isnotnull(ws_bill_customer_sk#56) Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#52] Right keys [1]: [ws_bill_customer_sk#56] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58] Input [7]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] (69) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#59, d_year#60] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#58] Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58, d_date_sk#59, d_year#60] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#57))] @@ -426,10 +426,10 @@ Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum# Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] Functions [1]: [sum(UnscaledValue(ws_net_paid#57))] @@ -440,13 +440,13 @@ Results [2]: [c_customer_id#39 AS customer_id#63, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#63, year_total#64] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#63] Join type: Inner Join condition: (CASE WHEN (year_total#51 > 0.00) THEN (year_total#64 / year_total#51) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) -(78) Project [codegen id : 16] +(78) Project [codegen id : 12] Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51, customer_id#63, year_total#64] @@ -458,7 +458,7 @@ Arguments: 100, [customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometFilter (81) +- CometNativeScan parquet spark_catalog.default.date_dim (80) @@ -474,7 +474,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (83) BroadcastExchange @@ -483,7 +483,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (87) -+- * CometColumnarToRow (86) ++- CometNativeColumnarToRow (86) +- CometFilter (85) +- CometNativeScan parquet spark_catalog.default.date_dim (84) @@ -499,7 +499,7 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_year#28] Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) -(86) CometColumnarToRow [codegen id : 1] +(86) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_year#28] (87) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt index 0d59f7b833..5d218998be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -23,23 +23,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -49,24 +49,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -76,19 +76,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -98,7 +98,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/simplified.txt index f488e18c7c..01e22b7b4a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -7,17 +7,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] @@ -30,69 +30,65 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -102,25 +98,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/explain.txt index 649b5dc4d5..046c271650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_year#26] Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_year#26] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/simplified.txt index 72e39422e0..9ec698fdca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/simplified.txt @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt index 649b5dc4d5..046c271650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_year#26] Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_year#26] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 72e39422e0..9ec698fdca 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 @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/explain.txt index abecd7ac24..d4b4089cf1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (127) +CometNativeColumnarToRow (127) +- CometTakeOrderedAndProject (126) +- CometProject (125) +- CometSortMergeJoin (124) @@ -25,7 +25,7 @@ : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -136,10 +136,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Condition : isnotnull(cs_item_sk#1) @@ -158,33 +158,33 @@ Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarc Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (8) BroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (11) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#13, d_year#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] @@ -236,36 +236,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(24) ColumnarToRow [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) (26) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#22] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] (29) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#32, d_year#33] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] @@ -317,36 +317,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 9] +(42) ColumnarToRow [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -(43) Filter [codegen id : 9] +(43) Filter [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Condition : isnotnull(ws_item_sk#41) (44) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -(45) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_item_sk#41] Right keys [1]: [i_item_sk#46] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(46) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] (47) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#51, d_year#52] -(48) BroadcastHashJoin [codegen id : 9] +(48) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#45] Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(49) Project [codegen id : 9] +(49) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] @@ -443,36 +443,36 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 12] +(70) ColumnarToRow [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -(71) Filter [codegen id : 12] +(71) Filter [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] Condition : isnotnull(cs_item_sk#64) (72) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -(73) BroadcastHashJoin [codegen id : 12] +(73) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_item_sk#64] Right keys [1]: [i_item_sk#70] Join type: Inner Join condition: None -(74) Project [codegen id : 12] +(74) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] (75) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#75, d_year#76] -(76) BroadcastHashJoin [codegen id : 12] +(76) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#68] Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(77) Project [codegen id : 12] +(77) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] @@ -508,36 +508,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 15] +(85) ColumnarToRow [codegen id : 5] Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -(86) Filter [codegen id : 15] +(86) Filter [codegen id : 5] Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] Condition : isnotnull(ss_item_sk#81) (87) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] -(88) BroadcastHashJoin [codegen id : 15] +(88) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#81] Right keys [1]: [i_item_sk#86] Join type: Inner Join condition: None -(89) Project [codegen id : 15] +(89) Project [codegen id : 5] Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] Input [10]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] (90) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#91, d_year#92] -(91) BroadcastHashJoin [codegen id : 15] +(91) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#85] Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(92) Project [codegen id : 15] +(92) Project [codegen id : 5] Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] Input [11]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_date_sk#91, d_year#92] @@ -573,36 +573,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 18] +(100) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -(101) Filter [codegen id : 18] +(101) Filter [codegen id : 6] Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] Condition : isnotnull(ws_item_sk#97) (102) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] -(103) BroadcastHashJoin [codegen id : 18] +(103) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#97] Right keys [1]: [i_item_sk#102] Join type: Inner Join condition: None -(104) Project [codegen id : 18] +(104) Project [codegen id : 6] Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] Input [10]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] (105) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#107, d_year#108] -(106) BroadcastHashJoin [codegen id : 18] +(106) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#101] Right keys [1]: [d_date_sk#107] Join type: Inner Join condition: None -(107) Project [codegen id : 18] +(107) Project [codegen id : 6] Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] Input [11]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_date_sk#107, d_year#108] @@ -688,14 +688,14 @@ Arguments: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#120 ASC NULLS FIRST], output=[prev_year#116,year#117,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#118,curr_yr_cnt#119,sales_cnt_diff#120,sales_amt_diff#121]), [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], 100, 0, [sales_cnt_diff#120 ASC NULLS FIRST], [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] -(127) CometColumnarToRow [codegen id : 19] +(127) CometNativeColumnarToRow Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (131) -+- * CometColumnarToRow (130) ++- CometNativeColumnarToRow (130) +- CometFilter (129) +- CometNativeScan parquet spark_catalog.default.date_dim (128) @@ -711,7 +711,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (131) BroadcastExchange @@ -724,7 +724,7 @@ Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN d Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 BroadcastExchange (135) -+- * CometColumnarToRow (134) ++- CometNativeColumnarToRow (134) +- CometFilter (133) +- CometNativeScan parquet spark_catalog.default.date_dim (132) @@ -740,7 +740,7 @@ ReadSchema: struct Input [2]: [d_date_sk#75, d_year#76] Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) -(134) CometColumnarToRow [codegen id : 1] +(134) CometNativeColumnarToRow Input [2]: [d_date_sk#75, d_year#76] (135) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt index 9f6954be38..42106a0ffd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometSortMergeJoin @@ -25,16 +25,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -55,12 +55,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -81,12 +81,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -117,16 +117,16 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -147,12 +147,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -173,12 +173,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/simplified.txt index aaf98f4b09..84b015b466 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/simplified.txt @@ -1,173 +1,165 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - 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 [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 [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] +CometNativeColumnarToRow + CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + 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 [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 [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] - 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] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometExchange [cr_order_number,cr_item_sk] #7 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (2) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometExchange [sr_ticket_number,sr_item_sk] #9 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (3) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometExchange [wr_order_number,wr_item_sk] #11 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 + 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] #14 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] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (4) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (5) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (6) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/explain.txt index 7afb1e3be6..0ea1f4907b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/simplified.txt index d0520c8b63..0e2370eea4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/simplified.txt @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt index 7afb1e3be6..0ea1f4907b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange 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 d0520c8b63..0e2370eea4 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 @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/explain.txt index f04f5a8a5a..435ede764a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometTakeOrderedAndProject (34) +- CometHashAggregate (33) +- CometExchange (32) @@ -195,6 +195,6 @@ Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/extended.txt index cbb627c528..cfa19a28b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/simplified.txt index abb482c3d5..f502dc6f07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/simplified.txt @@ -1,37 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - 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 [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] - CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] - CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedExchange [i_item_sk,i_category] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] - CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedExchange [i_item_sk,i_category] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #3 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + 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 [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] + CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometProject [i_category] [i_item_sk,i_category] + CometFilter [i_item_sk,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/explain.txt index ae530b4900..d68b43b40a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (92) +- * HashAggregate (91) - +- * CometColumnarToRow (90) + +- CometNativeColumnarToRow (90) +- CometColumnarExchange (89) +- * HashAggregate (88) +- * Expand (87) @@ -9,7 +9,7 @@ TakeOrderedAndProject (92) :- * Project (32) : +- * BroadcastHashJoin LeftOuter BuildRight (31) : :- * HashAggregate (16) - : : +- * CometColumnarToRow (15) + : : +- CometNativeColumnarToRow (15) : : +- CometColumnarExchange (14) : : +- * HashAggregate (13) : : +- * Project (12) @@ -21,12 +21,12 @@ TakeOrderedAndProject (92) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (10) - : : +- * CometColumnarToRow (9) + : : +- CometNativeColumnarToRow (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (30) : +- * HashAggregate (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometColumnarExchange (27) : +- * HashAggregate (26) : +- * Project (25) @@ -42,7 +42,7 @@ TakeOrderedAndProject (92) : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) : :- BroadcastExchange (42) : : +- * HashAggregate (41) - : : +- * CometColumnarToRow (40) + : : +- CometNativeColumnarToRow (40) : : +- CometColumnarExchange (39) : : +- * HashAggregate (38) : : +- * Project (37) @@ -51,7 +51,7 @@ TakeOrderedAndProject (92) : : : +- Scan parquet spark_catalog.default.catalog_sales (33) : : +- ReusedExchange (35) : +- * HashAggregate (51) - : +- * CometColumnarToRow (50) + : +- CometNativeColumnarToRow (50) : +- CometColumnarExchange (49) : +- * HashAggregate (48) : +- * Project (47) @@ -62,7 +62,7 @@ TakeOrderedAndProject (92) +- * Project (85) +- * BroadcastHashJoin LeftOuter BuildRight (84) :- * HashAggregate (69) - : +- * CometColumnarToRow (68) + : +- CometNativeColumnarToRow (68) : +- CometColumnarExchange (67) : +- * HashAggregate (66) : +- * Project (65) @@ -74,12 +74,12 @@ TakeOrderedAndProject (92) : : : +- Scan parquet spark_catalog.default.web_sales (54) : : +- ReusedExchange (57) : +- BroadcastExchange (63) - : +- * CometColumnarToRow (62) + : +- CometNativeColumnarToRow (62) : +- CometFilter (61) : +- CometNativeScan parquet spark_catalog.default.web_page (60) +- BroadcastExchange (83) +- * HashAggregate (82) - +- * CometColumnarToRow (81) + +- CometNativeColumnarToRow (81) +- CometColumnarExchange (80) +- * HashAggregate (79) +- * Project (78) @@ -101,23 +101,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] @@ -132,24 +132,24 @@ ReadSchema: struct Input [1]: [s_store_sk#7] Condition : isnotnull(s_store_sk#7) -(9) CometColumnarToRow [codegen id : 2] +(9) CometNativeColumnarToRow Input [1]: [s_store_sk#7] (10) BroadcastExchange Input [1]: [s_store_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(12) Project [codegen id : 3] +(12) Project [codegen id : 1] Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Keys [1]: [s_store_sk#7] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] @@ -160,10 +160,10 @@ Results [3]: [s_store_sk#7, sum#10, sum#11] Input [3]: [s_store_sk#7, sum#10, sum#11] Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(15) CometColumnarToRow [codegen id : 8] +(15) CometNativeColumnarToRow Input [3]: [s_store_sk#7, sum#10, sum#11] -(16) HashAggregate [codegen id : 8] +(16) HashAggregate [codegen id : 4] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] @@ -178,40 +178,40 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Condition : isnotnull(sr_store_sk#16) (20) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#20] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] (23) ReusedExchange [Reuses operator id: 10] Output [1]: [s_store_sk#21] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_store_sk#16] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Keys [1]: [s_store_sk#21] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] @@ -222,10 +222,10 @@ Results [3]: [s_store_sk#21, sum#24, sum#25] Input [3]: [s_store_sk#21, sum#24, sum#25] Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#21, sum#24, sum#25] -(29) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 3] Input [3]: [s_store_sk#21, sum#24, sum#25] Keys [1]: [s_store_sk#21] Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] @@ -236,13 +236,13 @@ Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26 Input [3]: [s_store_sk#21, returns#28, profit_loss#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#7] Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 4] Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33] Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] @@ -253,23 +253,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 5] Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] (35) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#38] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#37] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 5] Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] -(38) HashAggregate [codegen id : 10] +(38) HashAggregate [codegen id : 5] Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Keys [1]: [cs_call_center_sk#34] Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] @@ -280,10 +280,10 @@ Results [3]: [cs_call_center_sk#34, sum#41, sum#42] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometColumnarToRow [codegen id : 11] +(40) CometNativeColumnarToRow Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 6] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Keys [1]: [cs_call_center_sk#34] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] @@ -301,23 +301,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] +(44) ColumnarToRow [codegen id : 7] Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] (45) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#50] -(46) BroadcastHashJoin [codegen id : 13] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cr_returned_date_sk#49] Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(47) Project [codegen id : 13] +(47) Project [codegen id : 7] Output [2]: [cr_return_amount#47, cr_net_loss#48] Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] -(48) HashAggregate [codegen id : 13] +(48) HashAggregate [codegen id : 7] Input [2]: [cr_return_amount#47, cr_net_loss#48] Keys: [] Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] @@ -328,7 +328,7 @@ Results [2]: [sum#53, sum#54] Input [2]: [sum#53, sum#54] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) CometColumnarToRow +(50) CometNativeColumnarToRow Input [2]: [sum#53, sum#54] (51) HashAggregate @@ -338,11 +338,11 @@ Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_ne Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] -(52) BroadcastNestedLoopJoin [codegen id : 14] +(52) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 8] Output [5]: [sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61] Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] @@ -354,23 +354,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 17] +(55) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -(56) Filter [codegen id : 17] +(56) Filter [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] Condition : isnotnull(ws_web_page_sk#62) (57) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#66] -(58) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#65] Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(59) Project [codegen id : 9] Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] @@ -385,24 +385,24 @@ ReadSchema: struct Input [1]: [wp_web_page_sk#67] Condition : isnotnull(wp_web_page_sk#67) -(62) CometColumnarToRow [codegen id : 16] +(62) CometNativeColumnarToRow Input [1]: [wp_web_page_sk#67] (63) BroadcastExchange Input [1]: [wp_web_page_sk#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(64) BroadcastHashJoin [codegen id : 17] +(64) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_web_page_sk#62] Right keys [1]: [wp_web_page_sk#67] Join type: Inner Join condition: None -(65) Project [codegen id : 17] +(65) Project [codegen id : 9] Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 9] Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Keys [1]: [wp_web_page_sk#67] Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] @@ -413,10 +413,10 @@ Results [3]: [wp_web_page_sk#67, sum#70, sum#71] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometColumnarToRow [codegen id : 22] +(68) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -(69) HashAggregate [codegen id : 22] +(69) HashAggregate [codegen id : 12] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Keys [1]: [wp_web_page_sk#67] Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] @@ -431,40 +431,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -(72) Filter [codegen id : 20] +(72) Filter [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] Condition : isnotnull(wr_web_page_sk#76) (73) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#80] -(74) BroadcastHashJoin [codegen id : 20] +(74) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_returned_date_sk#79] Right keys [1]: [d_date_sk#80] Join type: Inner Join condition: None -(75) Project [codegen id : 20] +(75) Project [codegen id : 10] Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] (76) ReusedExchange [Reuses operator id: 63] Output [1]: [wp_web_page_sk#81] -(77) BroadcastHashJoin [codegen id : 20] +(77) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_web_page_sk#76] Right keys [1]: [wp_web_page_sk#81] Join type: Inner Join condition: None -(78) Project [codegen id : 20] +(78) Project [codegen id : 10] Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -(79) HashAggregate [codegen id : 20] +(79) HashAggregate [codegen id : 10] Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Keys [1]: [wp_web_page_sk#81] Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] @@ -475,10 +475,10 @@ Results [3]: [wp_web_page_sk#81, sum#84, sum#85] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(81) CometColumnarToRow [codegen id : 21] +(81) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -(82) HashAggregate [codegen id : 21] +(82) HashAggregate [codegen id : 11] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Keys [1]: [wp_web_page_sk#81] Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] @@ -489,23 +489,23 @@ Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77) Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 22] +(84) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wp_web_page_sk#67] Right keys [1]: [wp_web_page_sk#81] Join type: LeftOuter Join condition: None -(85) Project [codegen id : 22] +(85) Project [codegen id : 12] Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93] Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] (86) Union -(87) Expand [codegen id : 23] +(87) Expand [codegen id : 13] Input [5]: [sales#14, returns#30, profit#31, channel#32, id#33] Arguments: [[sales#14, returns#30, profit#31, channel#32, id#33, 0], [sales#14, returns#30, profit#31, channel#32, null, 1], [sales#14, returns#30, profit#31, null, null, 3]], [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] -(88) HashAggregate [codegen id : 23] +(88) HashAggregate [codegen id : 13] Input [6]: [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] Keys [3]: [channel#94, id#95, spark_grouping_id#96] Functions [3]: [partial_sum(sales#14), partial_sum(returns#30), partial_sum(profit#31)] @@ -516,10 +516,10 @@ Results [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] Arguments: hashpartitioning(channel#94, id#95, spark_grouping_id#96, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(90) CometColumnarToRow [codegen id : 24] +(90) CometNativeColumnarToRow Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -(91) HashAggregate [codegen id : 24] +(91) HashAggregate [codegen id : 14] Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] Keys [3]: [channel#94, id#95, spark_grouping_id#96] Functions [3]: [sum(sales#14), sum(returns#30), sum(profit#31)] @@ -534,7 +534,7 @@ Arguments: 100, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (97) -+- * CometColumnarToRow (96) ++- CometNativeColumnarToRow (96) +- CometProject (95) +- CometFilter (94) +- CometNativeScan parquet spark_catalog.default.date_dim (93) @@ -555,7 +555,7 @@ Condition : (((isnotnull(d_date#115) AND (d_date#115 >= 2000-08-03)) AND (d_date Input [2]: [d_date_sk#6, d_date#115] Arguments: [d_date_sk#6], [d_date_sk#6] -(96) CometColumnarToRow [codegen id : 1] +(96) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (97) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt index d12b8dde24..5d2ca1a9d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -8,7 +8,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -20,22 +20,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -47,19 +47,19 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -68,12 +68,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -82,14 +82,14 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- BroadcastHashJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -101,17 +101,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -123,12 +123,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/simplified.txt index 4256e90759..5a732180f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/simplified.txt @@ -1,22 +1,22 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) + WholeStageCodegen (14) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) + WholeStageCodegen (13) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (8) + WholeStageCodegen (4) Project [sales,returns,profit,profit_loss,s_store_sk] BroadcastHashJoin [s_store_sk,s_store_sk] HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_sk] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] Project [ss_ext_sales_price,ss_net_profit,s_store_sk] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -28,29 +28,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_sk] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] Project [sr_return_amt,sr_net_loss,s_store_sk] BroadcastHashJoin [sr_store_sk,s_store_sk] @@ -65,17 +61,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [sales,returns,profit,profit_loss,cs_call_center_sk] BroadcastNestedLoopJoin InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) + WholeStageCodegen (6) HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cs_call_center_sk] #8 - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -86,10 +82,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [d_date_sk] #3 HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] Project [cr_return_amount,cr_net_loss] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] @@ -99,14 +95,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) + WholeStageCodegen (12) Project [sales,returns,profit,profit_loss,wp_web_page_sk] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wp_web_page_sk] #10 - WholeStageCodegen (17) + WholeStageCodegen (9) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] @@ -121,19 +117,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometNativeColumnarToRow + CometFilter [wp_web_page_sk] + CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter BroadcastExchange #12 - WholeStageCodegen (21) + WholeStageCodegen (11) HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wp_web_page_sk] #13 - WholeStageCodegen (20) + WholeStageCodegen (10) HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] Project [wr_return_amt,wr_net_loss,wp_web_page_sk] BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/explain.txt index bb7ed0a50c..9155629835 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/explain.txt @@ -477,7 +477,7 @@ Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (90) -+- * CometColumnarToRow (89) ++- CometNativeColumnarToRow (89) +- CometProject (88) +- CometFilter (87) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) @@ -498,7 +498,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(89) CometColumnarToRow [codegen id : 1] +(89) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (90) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt index ed8a9e38ca..a16105c248 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/simplified.txt index a1243769e5..94722e1db2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/simplified.txt @@ -25,12 +25,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt index bb7ed0a50c..9155629835 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt @@ -477,7 +477,7 @@ Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (90) -+- * CometColumnarToRow (89) ++- CometNativeColumnarToRow (89) +- CometProject (88) +- CometFilter (87) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) @@ -498,7 +498,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(89) CometColumnarToRow [codegen id : 1] +(89) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (90) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt index ed8a9e38ca..a16105c248 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a1243769e5..94722e1db2 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 @@ -25,12 +25,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/explain.txt index 4dc7ac380a..928636dfa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (70) +- * Project (69) - +- * CometColumnarToRow (68) + +- CometNativeColumnarToRow (68) +- CometSortMergeJoin (67) :- CometProject (45) : +- CometSortMergeJoin (44) @@ -372,7 +372,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner -(68) CometColumnarToRow [codegen id : 4] +(68) CometNativeColumnarToRow Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] (69) Project [codegen id : 4] @@ -387,7 +387,7 @@ Arguments: 100, [ratio#61 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -403,7 +403,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (74) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt index 65256f2124..af4b7000b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSortMergeJoin :- CometProject : +- CometSortMergeJoin @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/simplified.txt index ebd0840eec..5419c4229c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (4) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] @@ -24,11 +24,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/explain.txt index 3d3ea5ab86..e4118a724e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/simplified.txt index 77c05217f4..67056a3026 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/simplified.txt @@ -21,11 +21,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt index 3d3ea5ab86..e4118a724e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort 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 77c05217f4..67056a3026 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 @@ -21,11 +21,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/explain.txt index 78ed64abe7..0cdeaf082a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (32) +- * Project (31) +- * BroadcastHashJoin Inner BuildRight (30) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -17,17 +17,17 @@ TakeOrderedAndProject (32) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.customer (25) @@ -41,23 +41,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#10] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] @@ -76,20 +76,20 @@ Condition : (((isnotnull(s_number_employees#12) AND (s_number_employees#12 >= 20 Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] Arguments: [s_store_sk#11, s_city#13], [s_store_sk#11, s_city#13] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [s_store_sk#11, s_city#13] (11) BroadcastExchange Input [2]: [s_store_sk#11, s_city#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11, s_city#13] @@ -108,24 +108,24 @@ Condition : (((hd_dep_count#15 = 6) OR (hd_vehicle_count#16 > 2)) AND isnotnull( Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] Arguments: [hd_demo_sk#14], [hd_demo_sk#14] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#14] (18) BroadcastExchange Input [1]: [hd_demo_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13, hd_demo_sk#14] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] @@ -136,10 +136,10 @@ Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] @@ -161,20 +161,20 @@ Condition : isnotnull(c_customer_sk#25) Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] Arguments: [c_customer_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#26, 20, true, false, true) AS c_first_name#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#27, 30, true, false, true) AS c_last_name#29] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] (29) BroadcastExchange Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [7]: [c_last_name#29, c_first_name#28, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#13] Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#28, c_last_name#29] @@ -186,7 +186,7 @@ Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((isnotnull(d_dow#32) AND (d_dow#32 = 1)) AND d_year#31 IN (1999,20 Input [3]: [d_date_sk#10, d_year#31, d_dow#32] Arguments: [d_date_sk#10], [d_date_sk#10] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt index 05bd194c34..f24d730d8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject +- Project +- BroadcastHashJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -16,27 +16,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/simplified.txt index 1ee7a286a4..48b20af3a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/simplified.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (6) + WholeStageCodegen (2) Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] @@ -20,35 +20,27 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + CometNativeColumnarToRow + CometProject [s_store_sk,s_city] + CometFilter [s_store_sk,s_number_employees,s_city] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/explain.txt index 2d2c6dc938..98be1506c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/explain.txt @@ -181,7 +181,7 @@ Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt index 012403275a..871cf47610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/simplified.txt index 5b252a906b..d9d0bedcfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/simplified.txt @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt index 2d2c6dc938..98be1506c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt @@ -181,7 +181,7 @@ Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/extended.txt index 012403275a..871cf47610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 5b252a906b..d9d0bedcfb 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 @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/explain.txt index 754a8871d1..d7f41206bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (44) +- * HashAggregate (43) - +- * CometColumnarToRow (42) + +- CometNativeColumnarToRow (42) +- CometColumnarExchange (41) +- * HashAggregate (40) +- * Project (39) @@ -15,12 +15,12 @@ TakeOrderedAndProject (44) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.store (7) +- BroadcastExchange (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometHashAggregate (35) +- CometExchange (34) +- CometHashAggregate (33) @@ -53,23 +53,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -88,20 +88,20 @@ Condition : (isnotnull(s_store_sk#6) AND isnotnull(staticinvoke(class org.apache Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] Arguments: [s_store_sk#6, s_store_name#7, s_zip#9], [s_store_sk#6, s_store_name#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#8, 10, true, false, true) AS s_zip#9] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] (11) BroadcastExchange Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#9] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#9] @@ -207,24 +207,24 @@ Input [1]: [ca_zip#11] Keys [1]: [ca_zip#11] Functions: [] -(36) CometColumnarToRow [codegen id : 3] +(36) CometNativeColumnarToRow Input [1]: [ca_zip#11] (37) BroadcastExchange Input [1]: [ca_zip#11] Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 4] +(38) BroadcastHashJoin [codegen id : 1] Left keys [1]: [substr(s_zip#9, 1, 2)] Right keys [1]: [substr(ca_zip#11, 1, 2)] Join type: Inner Join condition: None -(39) Project [codegen id : 4] +(39) Project [codegen id : 1] Output [2]: [ss_net_profit#2, s_store_name#7] Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#9, ca_zip#11] -(40) HashAggregate [codegen id : 4] +(40) HashAggregate [codegen id : 1] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] @@ -235,10 +235,10 @@ Results [2]: [s_store_name#7, sum#21] Input [2]: [s_store_name#7, sum#21] Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(42) CometColumnarToRow [codegen id : 5] +(42) CometNativeColumnarToRow Input [2]: [s_store_name#7, sum#21] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 2] Input [2]: [s_store_name#7, sum#21] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] @@ -253,7 +253,7 @@ Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometNativeScan parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2) Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt index 257b7f0d77..8c9eb7b8c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -14,22 +14,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/simplified.txt index 4df6d8f659..97b1393e3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] BroadcastHashJoin [s_zip,ca_zip] @@ -18,46 +18,40 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometNativeColumnarToRow + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter + CometNativeColumnarToRow + CometHashAggregate [ca_zip] + CometExchange [ca_zip] #5 CometHashAggregate [ca_zip] - CometExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #6 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #7 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #8 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometBroadcastHashJoin [ca_zip,ca_zip] + CometProject [ca_zip] [ca_zip] + CometFilter [ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_zip] + CometBroadcastExchange [ca_zip] #6 + CometProject [ca_zip] + CometFilter [ca_zip,cnt] + CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] + CometExchange [ca_zip] #7 + CometHashAggregate [ca_zip,count] + CometProject [ca_zip] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometProject [ca_zip] [ca_address_sk,ca_zip] + CometFilter [ca_address_sk,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [c_current_addr_sk] #8 + CometProject [c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/explain.txt index 075f17ea3d..c84da45c11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/explain.txt @@ -253,7 +253,7 @@ Input [2]: [s_store_name#9, sum(ss_net_profit)#23] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt index 8e04bbeba7..4d7a7d1ce7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/simplified.txt index d1542ab607..fc8379fb0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/simplified.txt @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt index 075f17ea3d..c84da45c11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt @@ -253,7 +253,7 @@ Input [2]: [s_store_name#9, sum(ss_net_profit)#23] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/extended.txt index 8e04bbeba7..4d7a7d1ce7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d1542ab607..fc8379fb0d 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 @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/explain.txt index f1e1a2ace1..5a669c7083 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (106) +CometNativeColumnarToRow (106) +- CometTakeOrderedAndProject (105) +- CometHashAggregate (104) +- CometExchange (103) @@ -581,14 +581,14 @@ Functions [3]: [sum(sales#74), sum(returns#75), sum(profit#76)] Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,id#90 ASC NULLS FIRST], output=[channel#89,id#90,sales#98,returns#99,profit#100]), [channel#89, id#90, sales#98, returns#99, profit#100], 100, 0, [channel#89 ASC NULLS FIRST, id#90 ASC NULLS FIRST], [channel#89, id#90, sales#98, returns#99, profit#100] -(106) CometColumnarToRow [codegen id : 4] +(106) CometNativeColumnarToRow Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (111) -+- * CometColumnarToRow (110) ++- CometNativeColumnarToRow (110) +- CometProject (109) +- CometFilter (108) +- CometNativeScan parquet spark_catalog.default.date_dim (107) @@ -609,7 +609,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(110) CometColumnarToRow [codegen id : 1] +(110) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (111) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt index 9d97e27318..6853afccad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/simplified.txt index 09e9224cef..3d30801890 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/simplified.txt @@ -1,124 +1,120 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - 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] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - 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] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #11 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - 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] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #15 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] + CometExchange [s_store_id] #2 + 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] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 + WholeStageCodegen (1) + Filter [ss_store_sk,ss_item_sk,ss_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #5 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #10 + 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] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #11 + WholeStageCodegen (2) + Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] + CometExchange [web_site_id] #14 + 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] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #15 + WholeStageCodegen (3) + Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt index f0a163bf69..ff987b999b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt @@ -576,7 +576,7 @@ Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -597,7 +597,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt index ae6fc19b60..dd2aed67bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt index 198646e0fd..9781eeab1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #5 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt index f0a163bf69..ff987b999b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt @@ -576,7 +576,7 @@ Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -597,7 +597,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/extended.txt index ae6fc19b60..dd2aed67bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 198646e0fd..9781eeab1c 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 @@ -26,12 +26,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #5 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/explain.txt index 098cf9fe54..f6bb15b7d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (54) : : +- * BroadcastHashJoin Inner BuildRight (38) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) @@ -20,18 +20,18 @@ TakeOrderedAndProject (54) : : : : : +- Scan parquet spark_catalog.default.catalog_returns (1) : : : : +- ReusedExchange (4) : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) + : : : +- CometNativeColumnarToRow (10) : : : +- CometProject (9) : : : +- CometFilter (8) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : : +- BroadcastExchange (37) : : +- * Filter (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) + : : +- CometNativeColumnarToRow (30) : : +- CometColumnarExchange (29) : : +- * HashAggregate (28) : : +- * Project (27) @@ -44,12 +44,12 @@ TakeOrderedAndProject (54) : : : +- ReusedExchange (22) : : +- ReusedExchange (25) : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometProject (42) : +- CometFilter (41) : +- CometNativeScan parquet spark_catalog.default.customer (40) +- BroadcastExchange (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometNativeScan parquet spark_catalog.default.customer_address (47) @@ -63,23 +63,23 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] @@ -98,24 +98,24 @@ Condition : (isnotnull(ca_address_sk#7) AND isnotnull(staticinvoke(class org.apa Input [2]: [ca_address_sk#7, ca_state#8] Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#9] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [ca_address_sk#7, ca_state#9] (11) BroadcastExchange Input [2]: [ca_address_sk#7, ca_state#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#9] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] @@ -126,17 +126,17 @@ Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 11] +(16) CometNativeColumnarToRow Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -(17) HashAggregate [codegen id : 11] +(17) HashAggregate [codegen id : 5] Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] -(18) Filter [codegen id : 11] +(18) Filter [codegen id : 5] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) @@ -148,40 +148,40 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#19), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 2] Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -(21) Filter [codegen id : 6] +(21) Filter [codegen id : 2] Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] Condition : isnotnull(cr_returning_addr_sk#17) (22) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(24) Project [codegen id : 2] Output [3]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19, d_date_sk#20] (25) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#21, ca_state#9] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returning_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 2] Output [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#9] Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, ca_address_sk#21, ca_state#9] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#9] Keys [2]: [cr_returning_customer_sk#16, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#18))] @@ -192,17 +192,17 @@ Results [3]: [cr_returning_customer_sk#16, ca_state#9, sum#23] Input [3]: [cr_returning_customer_sk#16, ca_state#9, sum#23] Arguments: hashpartitioning(cr_returning_customer_sk#16, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [3]: [cr_returning_customer_sk#16, ca_state#9, sum#23] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [3]: [cr_returning_customer_sk#16, ca_state#9, sum#23] Keys [2]: [cr_returning_customer_sk#16, ca_state#9] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))#12] Results [2]: [ca_state#9 AS ctr_state#24, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#18))#12,17,2) AS ctr_total_return#25] -(32) HashAggregate [codegen id : 7] +(32) HashAggregate [codegen id : 3] Input [2]: [ctr_state#24, ctr_total_return#25] Keys [1]: [ctr_state#24] Functions [1]: [partial_avg(ctr_total_return#25)] @@ -213,17 +213,17 @@ Results [3]: [ctr_state#24, sum#28, count#29] Input [3]: [ctr_state#24, sum#28, count#29] Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 8] +(34) CometNativeColumnarToRow Input [3]: [ctr_state#24, sum#28, count#29] -(35) HashAggregate [codegen id : 8] +(35) HashAggregate [codegen id : 4] Input [3]: [ctr_state#24, sum#28, count#29] Keys [1]: [ctr_state#24] Functions [1]: [avg(ctr_total_return#25)] Aggregate Attributes [1]: [avg(ctr_total_return#25)#30] Results [2]: [(avg(ctr_total_return#25)#30 * 1.2) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#24] -(36) Filter [codegen id : 8] +(36) Filter [codegen id : 4] Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) @@ -231,13 +231,13 @@ Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 11] +(38) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_state#14] Right keys [1]: [ctr_state#24] Join type: Inner Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) -(39) Project [codegen id : 11] +(39) Project [codegen id : 5] Output [2]: [ctr_customer_sk#13, ctr_total_return#15] Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_state#24] @@ -256,20 +256,20 @@ Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] Arguments: [c_customer_sk#32, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#38, c_current_addr_sk#34, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#35, 10, true, false, true) AS c_salutation#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#36, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#37, 30, true, false, true) AS c_last_name#41] -(43) CometColumnarToRow [codegen id : 9] +(43) CometNativeColumnarToRow Input [6]: [c_customer_sk#32, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41] (44) BroadcastExchange Input [6]: [c_customer_sk#32, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_customer_sk#13] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(46) Project [codegen id : 5] Output [6]: [ctr_total_return#15, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41] Input [8]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#32, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41] @@ -288,20 +288,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [12]: [ca_address_sk#42, ca_street_number#43, ca_street_name#44, ca_street_type#45, ca_suite_number#46, ca_city#47, ca_county#48, ca_state#49, ca_zip#50, ca_country#51, ca_gmt_offset#52, ca_location_type#53] Arguments: [ca_address_sk#42, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59], [ca_address_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#43, 10, true, false, true) AS ca_street_number#54, ca_street_name#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_type#45, 15, true, false, true) AS ca_street_type#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_suite_number#46, 10, true, false, true) AS ca_suite_number#56, ca_city#47, ca_county#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#49, 2, true, false, true) AS ca_state#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#50, 10, true, false, true) AS ca_zip#58, ca_country#51, ca_gmt_offset#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_location_type#53, 20, true, false, true) AS ca_location_type#59] -(50) CometColumnarToRow [codegen id : 10] +(50) CometNativeColumnarToRow Input [12]: [ca_address_sk#42, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59] (51) BroadcastExchange Input [12]: [ca_address_sk#42, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#34] Right keys [1]: [ca_address_sk#42] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(53) Project [codegen id : 5] Output [16]: [c_customer_id#38, c_salutation#39, c_first_name#40, c_last_name#41, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59, ctr_total_return#15] Input [18]: [ctr_total_return#15, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41, ca_address_sk#42, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59] @@ -313,7 +313,7 @@ Arguments: 100, [c_customer_id#38 ASC NULLS FIRST, c_salutation#39 ASC NULLS FIR Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (59) -+- * CometColumnarToRow (58) ++- CometNativeColumnarToRow (58) +- CometProject (57) +- CometFilter (56) +- CometNativeScan parquet spark_catalog.default.date_dim (55) @@ -334,7 +334,7 @@ Condition : ((isnotnull(d_year#60) AND (d_year#60 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#6, d_year#60] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(58) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (59) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt index 0e70ec42fa..841b2ca52d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -19,28 +19,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -52,22 +52,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/simplified.txt index da396ce8a9..fe57ceb6af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (5) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] @@ -8,10 +8,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] @@ -23,37 +23,33 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st Scan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (8) + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] @@ -70,17 +66,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometNativeColumnarToRow + CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/explain.txt index 35974f7924..219e9d8414 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/explain.txt @@ -285,7 +285,7 @@ Input [16]: [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt index 9708802e79..731fcea912 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/simplified.txt index 1091e272cc..120f97f276 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt index 35974f7924..219e9d8414 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt @@ -285,7 +285,7 @@ Input [16]: [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/extended.txt index 9708802e79..731fcea912 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 1091e272cc..120f97f276 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/explain.txt index 93da88fcc1..12332a6d23 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (26) +CometNativeColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometColumnarExchange (23) @@ -11,7 +11,7 @@ : +- * BroadcastHashJoin Inner BuildRight (13) : :- * Project (11) : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -21,7 +21,7 @@ : : +- * ColumnarToRow (6) : : +- Scan parquet spark_catalog.default.inventory (5) : +- ReusedExchange (12) - +- * CometColumnarToRow (19) + +- CometNativeColumnarToRow (19) +- CometProject (18) +- CometFilter (17) +- CometNativeScan parquet spark_catalog.default.store_sales (16) @@ -42,7 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] (5) Scan parquet spark_catalog.default.inventory @@ -68,26 +68,26 @@ Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] Input [2]: [inv_item_sk#7, inv_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [inv_item_sk#7] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] (12) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#11] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] @@ -110,20 +110,20 @@ Condition : isnotnull(ss_item_sk#12) Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] Arguments: [ss_item_sk#12], [ss_item_sk#12] -(19) CometColumnarToRow +(19) CometNativeColumnarToRow Input [1]: [ss_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#12] Join type: Inner Join condition: None -(21) Project [codegen id : 4] +(21) Project [codegen id : 3] Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#12] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 3] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Functions: [] @@ -143,14 +143,14 @@ Functions: [] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] -(26) CometColumnarToRow [codegen id : 5] +(26) CometNativeColumnarToRow Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.date_dim (27) @@ -171,7 +171,7 @@ Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#14] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt index c86e7ba2cb..8abf5c1fb1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -10,7 +10,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -21,16 +21,16 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/simplified.txt index b60bf92c5b..b8b1c8b053 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/simplified.txt @@ -1,45 +1,41 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,ss_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [inv_item_sk,inv_date_sk] + Filter [inv_quantity_on_hand,inv_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + CometNativeColumnarToRow + CometProject [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/explain.txt index 698608e16d..9380337fed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt index 9fbb7893b8..2ab6dd2d0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/simplified.txt index 28d160a4a8..b92f067e53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt index 698608e16d..9380337fed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/extended.txt index 9fbb7893b8..2ab6dd2d0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt index 28d160a4a8..b92f067e53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/explain.txt index cae143b42b..8e52da4794 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (50) :- * Project (33) : +- * BroadcastHashJoin Inner BuildRight (32) : :- * HashAggregate (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- * HashAggregate (14) : : +- * Project (13) @@ -16,14 +16,14 @@ TakeOrderedAndProject (50) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_returns (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : +- ReusedExchange (11) : +- BroadcastExchange (31) : +- * HashAggregate (30) - : +- * CometColumnarToRow (29) + : +- CometNativeColumnarToRow (29) : +- CometColumnarExchange (28) : +- * HashAggregate (27) : +- * Project (26) @@ -37,7 +37,7 @@ TakeOrderedAndProject (50) : +- ReusedExchange (24) +- BroadcastExchange (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * Project (42) @@ -59,10 +59,10 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) @@ -81,37 +81,37 @@ Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache. Input [2]: [i_item_sk#5, i_item_id#6] Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [i_item_sk#5, i_item_id#7] (8) BroadcastExchange Input [2]: [i_item_sk#5, i_item_id#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [sr_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] (11) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#8] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [sr_returned_date_sk#3] Right keys [1]: [d_date_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [2]: [sr_return_quantity#2, i_item_id#7] Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [2]: [sr_return_quantity#2, i_item_id#7] Keys [1]: [i_item_id#7] Functions [1]: [partial_sum(sr_return_quantity#2)] @@ -122,10 +122,10 @@ Results [2]: [i_item_id#7, sum#10] Input [2]: [i_item_id#7, sum#10] Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 12] +(16) CometNativeColumnarToRow Input [2]: [i_item_id#7, sum#10] -(17) HashAggregate [codegen id : 12] +(17) HashAggregate [codegen id : 6] Input [2]: [i_item_id#7, sum#10] Keys [1]: [i_item_id#7] Functions [1]: [sum(sr_return_quantity#2)] @@ -140,40 +140,40 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#16), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 2] Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -(20) Filter [codegen id : 6] +(20) Filter [codegen id : 2] Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] Condition : isnotnull(cr_item_sk#14) (21) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#17, i_item_id#18] -(22) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_item_sk#14] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(23) Project [codegen id : 2] Output [3]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#18] Input [5]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16, i_item_sk#17, i_item_id#18] (24) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#19] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returned_date_sk#16] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(26) Project [codegen id : 2] Output [2]: [cr_return_quantity#15, i_item_id#18] Input [4]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#18, d_date_sk#19] -(27) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 2] Input [2]: [cr_return_quantity#15, i_item_id#18] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(cr_return_quantity#15)] @@ -184,10 +184,10 @@ Results [2]: [i_item_id#18, sum#21] Input [2]: [i_item_id#18, sum#21] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(29) CometColumnarToRow [codegen id : 7] +(29) CometNativeColumnarToRow Input [2]: [i_item_id#18, sum#21] -(30) HashAggregate [codegen id : 7] +(30) HashAggregate [codegen id : 3] Input [2]: [i_item_id#18, sum#21] Keys [1]: [i_item_id#18] Functions [1]: [sum(cr_return_quantity#15)] @@ -198,13 +198,13 @@ Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#15)#22 AS cr_it Input [2]: [item_id#23, cr_item_qty#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 12] +(32) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#23] Join type: Inner Join condition: None -(33) Project [codegen id : 12] +(33) Project [codegen id : 6] Output [3]: [item_id#12, sr_item_qty#13, cr_item_qty#24] Input [4]: [item_id#12, sr_item_qty#13, item_id#23, cr_item_qty#24] @@ -216,40 +216,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] +(35) ColumnarToRow [codegen id : 4] Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -(36) Filter [codegen id : 10] +(36) Filter [codegen id : 4] Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] Condition : isnotnull(wr_item_sk#25) (37) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#28, i_item_id#29] -(38) BroadcastHashJoin [codegen id : 10] +(38) BroadcastHashJoin [codegen id : 4] Left keys [1]: [wr_item_sk#25] Right keys [1]: [i_item_sk#28] Join type: Inner Join condition: None -(39) Project [codegen id : 10] +(39) Project [codegen id : 4] Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#29] Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#28, i_item_id#29] (40) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#30] -(41) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [wr_returned_date_sk#27] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 10] +(42) Project [codegen id : 4] Output [2]: [wr_return_quantity#26, i_item_id#29] Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#29, d_date_sk#30] -(43) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 4] Input [2]: [wr_return_quantity#26, i_item_id#29] Keys [1]: [i_item_id#29] Functions [1]: [partial_sum(wr_return_quantity#26)] @@ -260,10 +260,10 @@ Results [2]: [i_item_id#29, sum#32] Input [2]: [i_item_id#29, sum#32] Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 11] +(45) CometNativeColumnarToRow Input [2]: [i_item_id#29, sum#32] -(46) HashAggregate [codegen id : 11] +(46) HashAggregate [codegen id : 5] Input [2]: [i_item_id#29, sum#32] Keys [1]: [i_item_id#29] Functions [1]: [sum(wr_return_quantity#26)] @@ -274,13 +274,13 @@ Results [2]: [i_item_id#29 AS item_id#34, sum(wr_return_quantity#26)#33 AS wr_it Input [2]: [item_id#34, wr_item_qty#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 12] +(48) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#34] Join type: Inner Join condition: None -(49) Project [codegen id : 12] +(49) Project [codegen id : 6] Output [8]: [item_id#12, sr_item_qty#13, (((cast(sr_item_qty#13 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS sr_dev#36, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS cr_dev#37, wr_item_qty#35, (((cast(wr_item_qty#35 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS wr_dev#38, (cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as decimal(20,0)) / 3.0) AS average#39] Input [5]: [item_id#12, sr_item_qty#13, cr_item_qty#24, item_id#34, wr_item_qty#35] @@ -292,7 +292,7 @@ Arguments: 100, [item_id#12 ASC NULLS FIRST, sr_item_qty#13 ASC NULLS FIRST], [i Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (64) -+- * CometColumnarToRow (63) ++- CometNativeColumnarToRow (63) +- CometProject (62) +- CometBroadcastHashJoin (61) :- CometFilter (52) @@ -364,7 +364,7 @@ Arguments: [d_date#40], [d_date#41], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#40] Arguments: [d_date_sk#8], [d_date_sk#8] -(63) CometColumnarToRow [codegen id : 1] +(63) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (64) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt index 1cb87c7eb1..7616ffd2b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -30,12 +30,12 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter @@ -50,7 +50,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -62,12 +62,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -82,7 +82,7 @@ TakeOrderedAndProject : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -94,12 +94,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/simplified.txt index 0673c590f5..53a9745325 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - WholeStageCodegen (12) + WholeStageCodegen (6) Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] BroadcastHashJoin [item_id,item_id] Project [item_id,sr_item_qty,cr_item_qty] BroadcastHashJoin [item_id,item_id] HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,sr_return_quantity] [sum,sum] Project [sr_return_quantity,i_item_id] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -20,39 +20,35 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,cr_return_quantity] [sum,sum] Project [cr_return_quantity,i_item_id] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] @@ -69,12 +65,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (5) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #9 - WholeStageCodegen (10) + WholeStageCodegen (4) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] BroadcastHashJoin [wr_returned_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/explain.txt index 515e2540f7..2d5c14b3c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/explain.txt @@ -277,7 +277,7 @@ Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometFilter (51) @@ -349,7 +349,7 @@ Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (63) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt index 924b12cf01..61e40ee179 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/simplified.txt index d016a5ee2d..f54f284f87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/simplified.txt @@ -17,21 +17,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #5 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt index 515e2540f7..2d5c14b3c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt @@ -277,7 +277,7 @@ Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometFilter (51) @@ -349,7 +349,7 @@ Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (63) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/extended.txt index 924b12cf01..61e40ee179 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter 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 d016a5ee2d..f54f284f87 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 @@ -17,21 +17,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #5 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/explain.txt index 63140cfc2d..98e5328b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (33) +CometNativeColumnarToRow (33) +- CometTakeOrderedAndProject (32) +- CometProject (31) +- CometBroadcastHashJoin (30) @@ -185,6 +185,6 @@ Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 Input [3]: [customer_id#20, customername#21, c_customer_id#7] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [customer_id#20, customername#21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/extended.txt index b025c868d1..402322f69f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/simplified.txt index 9df82b0e98..73465fcf59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/simplified.txt @@ -1,35 +1,33 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 - CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] - CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] - CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] +CometNativeColumnarToRow + CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] + CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] + CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastExchange [ca_address_sk] #2 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_city] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [cd_demo_sk] #3 + CometFilter [cd_demo_sk] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + CometBroadcastExchange [ib_income_band_sk] #5 + CometProject [ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + CometProject [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/explain.txt index 204cd4ea7a..0f21e58ceb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (52) +- * HashAggregate (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- CometColumnarExchange (49) +- * HashAggregate (48) +- * Project (47) @@ -22,32 +22,32 @@ TakeOrderedAndProject (52) : : : : : : : +- * Filter (3) : : : : : : : +- * ColumnarToRow (2) : : : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : : : +- * CometColumnarToRow (8) + : : : : : : +- CometNativeColumnarToRow (8) : : : : : : +- CometProject (7) : : : : : : +- CometFilter (6) : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (5) : : : : : +- BroadcastExchange (14) - : : : : : +- * CometColumnarToRow (13) + : : : : : +- CometNativeColumnarToRow (13) : : : : : +- CometFilter (12) : : : : : +- CometNativeScan parquet spark_catalog.default.web_page (11) : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) + : : : : +- CometNativeColumnarToRow (20) : : : : +- CometProject (19) : : : : +- CometFilter (18) : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (17) : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) + : : : +- CometNativeColumnarToRow (27) : : : +- CometProject (26) : : : +- CometFilter (25) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (24) : : +- BroadcastExchange (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometProject (33) : : +- CometFilter (32) : : +- CometNativeScan parquet spark_catalog.default.customer_address (31) : +- ReusedExchange (38) +- BroadcastExchange (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.reason (41) @@ -87,16 +87,16 @@ Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -(8) CometColumnarToRow +(8) CometNativeColumnarToRow Input [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -(9) BroadcastHashJoin [codegen id : 8] +(9) BroadcastHashJoin [codegen id : 2] Left keys [2]: [ws_item_sk#1, ws_order_number#3] Right keys [2]: [wr_item_sk#9, wr_order_number#14] Join type: Inner Join condition: None -(10) Project [codegen id : 8] +(10) Project [codegen id : 2] Output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] @@ -111,20 +111,20 @@ ReadSchema: struct Input [1]: [wp_web_page_sk#18] Condition : isnotnull(wp_web_page_sk#18) -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [1]: [wp_web_page_sk#18] (14) BroadcastExchange Input [1]: [wp_web_page_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 8] +(15) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_web_page_sk#2] Right keys [1]: [wp_web_page_sk#18] Join type: Inner Join condition: None -(16) Project [codegen id : 8] +(16) Project [codegen id : 2] Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] @@ -143,20 +143,20 @@ Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(staticinvoke(class org.apa Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] (21) BroadcastExchange Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 8] +(22) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_refunded_cdemo_sk#10] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) -(23) Project [codegen id : 8] +(23) Project [codegen id : 2] Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] @@ -175,20 +175,20 @@ Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(staticinvoke(class org.apac Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true) AS cd_education_status#28] -(27) CometColumnarToRow [codegen id : 4] +(27) CometNativeColumnarToRow Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] (28) BroadcastExchange Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, string, true], input[2, string, true]),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 2] Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23] Right keys [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] Join type: Inner Join condition: None -(30) Project [codegen id : 8] +(30) Project [codegen id : 2] Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] @@ -207,33 +207,33 @@ Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) AS ca_state#32] -(34) CometColumnarToRow [codegen id : 5] +(34) CometNativeColumnarToRow Input [2]: [ca_address_sk#29, ca_state#32] (35) BroadcastExchange Input [2]: [ca_address_sk#29, ca_state#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_refunded_addr_sk#11] Right keys [1]: [ca_address_sk#29] Join type: Inner Join condition: ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) -(37) Project [codegen id : 8] +(37) Project [codegen id : 2] Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] (38) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#33] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#7] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(40) Project [codegen id : 8] +(40) Project [codegen id : 2] Output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] @@ -252,24 +252,24 @@ Condition : isnotnull(r_reason_sk#34) Input [2]: [r_reason_sk#34, r_reason_desc#35] Arguments: [r_reason_sk#34, r_reason_desc#36], [r_reason_sk#34, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#35, 100, true, false, true) AS r_reason_desc#36] -(44) CometColumnarToRow [codegen id : 7] +(44) CometNativeColumnarToRow Input [2]: [r_reason_sk#34, r_reason_desc#36] (45) BroadcastExchange Input [2]: [r_reason_sk#34, r_reason_desc#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 8] +(46) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_reason_sk#13] Right keys [1]: [r_reason_sk#34] Join type: Inner Join condition: None -(47) Project [codegen id : 8] +(47) Project [codegen id : 2] Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#34, r_reason_desc#36] -(48) HashAggregate [codegen id : 8] +(48) HashAggregate [codegen id : 2] Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] Keys [1]: [r_reason_desc#36] Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] @@ -280,10 +280,10 @@ Results [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, coun Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] Arguments: hashpartitioning(r_reason_desc#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) CometColumnarToRow [codegen id : 9] +(50) CometNativeColumnarToRow Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] -(51) HashAggregate [codegen id : 9] +(51) HashAggregate [codegen id : 3] Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] Keys [1]: [r_reason_desc#36] Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] @@ -298,7 +298,7 @@ Arguments: 100, [substr(r_reason_desc, 1, 20)#52 ASC NULLS FIRST, avg(ws_quantit Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#33, d_year#56] Arguments: [d_date_sk#33], [d_date_sk#33] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt index 7f8b90a86c..aed7b365be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -23,40 +23,40 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_page : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/simplified.txt index 68d46e608d..e16d15b6c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [r_reason_desc] #1 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] BroadcastHashJoin [wr_reason_sk,r_reason_sk] @@ -29,55 +29,43 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometNativeColumnarToRow + CometFilter [wp_web_page_sk] + CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter + CometNativeColumnarToRow CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #7 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + CometNativeColumnarToRow + CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] + CometFilter [r_reason_sk,r_reason_desc] + CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/explain.txt index c263f03d76..25fbead789 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/explain.txt @@ -278,7 +278,7 @@ Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunde Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -299,7 +299,7 @@ Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#33, d_year#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt index e2383394d1..5c782b95f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/simplified.txt index dfcca32f03..1e7b26f922 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/simplified.txt @@ -24,12 +24,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt index c263f03d76..25fbead789 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt @@ -278,7 +278,7 @@ Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunde Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -299,7 +299,7 @@ Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#33, d_year#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/extended.txt index e2383394d1..5c782b95f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 dfcca32f03..1e7b26f922 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 @@ -24,12 +24,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/explain.txt index 3f745e99c5..1aa03c12e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (24) +- * Project (23) +- Window (22) - +- * CometColumnarToRow (21) + +- CometNativeColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometColumnarExchange (16) +- * HashAggregate (15) +- * Expand (14) @@ -19,7 +19,7 @@ TakeOrderedAndProject (24) : : +- Scan parquet spark_catalog.default.web_sales (1) : +- ReusedExchange (4) +- BroadcastExchange (11) - +- * CometColumnarToRow (10) + +- CometNativeColumnarToRow (10) +- CometProject (9) +- CometFilter (8) +- CometNativeScan parquet spark_catalog.default.item (7) @@ -33,23 +33,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ws_item_sk#1, ws_net_paid#2] Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] @@ -68,28 +68,28 @@ Condition : isnotnull(i_item_sk#6) Input [3]: [i_item_sk#6, i_class#7, i_category#8] Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#6, i_class#9, i_category#10] (11) BroadcastExchange Input [3]: [i_item_sk#6, i_class#9, i_category#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [ws_net_paid#2, i_category#10, i_class#9] Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] -(14) Expand [codegen id : 3] +(14) Expand [codegen id : 1] Input [3]: [ws_net_paid#2, i_category#10, i_class#9] Arguments: [[ws_net_paid#2, i_category#10, i_class#9, 0], [ws_net_paid#2, i_category#10, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 1] Input [4]: [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] @@ -100,10 +100,10 @@ Results [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] Arguments: hashpartitioning(i_category#11, i_class#12, spark_grouping_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] @@ -118,14 +118,14 @@ Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(21) CometColumnarToRow [codegen id : 5] +(21) CometNativeColumnarToRow Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] (22) Window Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] -(23) Project [codegen id : 6] +(23) Project [codegen id : 3] Output [5]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] Input [8]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] @@ -137,7 +137,7 @@ Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ Condition : (((isnotnull(d_month_seq#23) AND (d_month_seq#23 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#23] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt index 41081debd9..a60a3158a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -18,17 +18,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/simplified.txt index 0b24fe234a..be50cc4b5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/simplified.txt @@ -1,44 +1,38 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] + Expand [ws_net_paid,i_category,i_class] + Project [ws_net_paid,i_category,i_class] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/explain.txt index 1af5e2612c..c899fb891f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/explain.txt @@ -138,7 +138,7 @@ Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -159,7 +159,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt index dfc810b108..6444b727e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/simplified.txt index 29e2d72920..d1e99b5c9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt index 1af5e2612c..c899fb891f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt @@ -138,7 +138,7 @@ Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -159,7 +159,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt index dfc810b108..6444b727e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 29e2d72920..d1e99b5c9b 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 @@ -20,12 +20,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/explain.txt index b3086b072a..d6dacf8406 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == * HashAggregate (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometColumnarExchange (50) +- * HashAggregate (49) +- * Project (48) +- * BroadcastHashJoin LeftAnti BuildRight (47) :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (17) + : :- CometNativeColumnarToRow (17) : : +- CometHashAggregate (16) : : +- CometColumnarExchange (15) : : +- * HashAggregate (14) @@ -19,12 +19,12 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer (7) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometHashAggregate (29) : +- CometColumnarExchange (28) : +- * HashAggregate (27) @@ -38,7 +38,7 @@ : : +- ReusedExchange (21) : +- ReusedExchange (24) +- BroadcastExchange (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometHashAggregate (44) +- CometColumnarExchange (43) +- * HashAggregate (42) @@ -61,23 +61,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) (4) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#4, d_date#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#2] Right keys [1]: [d_date_sk#4] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, d_date#5] Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] @@ -96,24 +96,24 @@ Condition : isnotnull(c_customer_sk#6) Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#7, 20, true, false, true) AS c_first_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#8, 30, true, false, true) AS c_last_name#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] (11) BroadcastExchange Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [c_last_name#10, c_first_name#9, d_date#5] Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [c_last_name#10, c_first_name#9, d_date#5] Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] Functions: [] @@ -129,7 +129,7 @@ Input [3]: [c_last_name#10, c_first_name#9, d_date#5] Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] Functions: [] -(17) CometColumnarToRow [codegen id : 12] +(17) CometNativeColumnarToRow Input [3]: [c_last_name#10, c_first_name#9, d_date#5] (18) Scan parquet spark_catalog.default.catalog_sales @@ -140,40 +140,40 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -(20) Filter [codegen id : 6] +(20) Filter [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Condition : isnotnull(cs_bill_customer_sk#11) (21) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#13, d_date#14] -(22) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(23) Project [codegen id : 2] Output [2]: [cs_bill_customer_sk#11, d_date#14] Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] (24) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_bill_customer_sk#11] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(26) Project [codegen id : 2] Output [3]: [c_last_name#17, c_first_name#16, d_date#14] Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] -(27) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 2] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] @@ -189,14 +189,14 @@ Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (31) BroadcastExchange Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 12] +(32) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join type: LeftAnti @@ -210,40 +210,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(35) Filter [codegen id : 10] +(35) Filter [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Condition : isnotnull(ws_bill_customer_sk#18) (36) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#20, d_date#21] -(37) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(38) Project [codegen id : 3] Output [2]: [ws_bill_customer_sk#18, d_date#21] Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] (39) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_bill_customer_sk#18] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 3] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 3] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] @@ -259,24 +259,24 @@ Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] -(45) CometColumnarToRow [codegen id : 11] +(45) CometNativeColumnarToRow Input [3]: [c_last_name#24, c_first_name#23, d_date#21] (46) BroadcastExchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 12] +(47) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] Join type: LeftAnti Join condition: None -(48) Project [codegen id : 12] +(48) Project [codegen id : 4] Output: [] Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -(49) HashAggregate [codegen id : 12] +(49) HashAggregate [codegen id : 4] Input: [] Keys: [] Functions [1]: [partial_count(1)] @@ -287,10 +287,10 @@ Results [1]: [count#26] Input [1]: [count#26] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) CometColumnarToRow [codegen id : 13] +(51) CometNativeColumnarToRow Input [1]: [count#26] -(52) HashAggregate [codegen id : 13] +(52) HashAggregate [codegen id : 5] Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] @@ -301,7 +301,7 @@ Results [1]: [count(1)#27 AS count(1)#28] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -322,7 +322,7 @@ Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_mo Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt index 638b8865e0..7a9e62d57c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt @@ -1,11 +1,11 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -18,22 +18,22 @@ HashAggregate : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometColumnarExchange : +- HashAggregate @@ -46,17 +46,17 @@ HashAggregate : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate @@ -69,12 +69,12 @@ HashAggregate : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/simplified.txt index afc302e779..25e4104be0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (13) +WholeStageCodegen (5) HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (12) + WholeStageCodegen (4) HashAggregate [count,count] Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [c_last_name,c_first_name,d_date] CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] BroadcastHashJoin [ss_customer_sk,c_customer_sk] @@ -24,63 +24,55 @@ WholeStageCodegen (13) Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter ReusedExchange [d_date_sk,d_date] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometNativeColumnarToRow + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometNativeColumnarToRow + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/explain.txt index fc6b12b834..0cfb76e0f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/explain.txt @@ -292,7 +292,7 @@ Results [1]: [count(1)#30 AS count(1)#31] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (56) -+- * CometColumnarToRow (55) ++- CometNativeColumnarToRow (55) +- CometProject (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -313,7 +313,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(55) CometColumnarToRow [codegen id : 1] +(55) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (56) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt index ddee139acf..0c51ae09a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ HashAggregate : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/simplified.txt index f687139735..e1555c4783 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt index fc6b12b834..0cfb76e0f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt @@ -292,7 +292,7 @@ Results [1]: [count(1)#30 AS count(1)#31] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (56) -+- * CometColumnarToRow (55) ++- CometNativeColumnarToRow (55) +- CometProject (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -313,7 +313,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(55) CometColumnarToRow [codegen id : 1] +(55) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (56) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/extended.txt index ddee139acf..0c51ae09a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/extended.txt @@ -17,7 +17,7 @@ HashAggregate : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt index f687139735..e1555c4783 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/explain.txt index 25b4c305f2..d2d4ce18d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/explain.txt @@ -6,7 +6,7 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) +: : : : : : :- CometNativeColumnarToRow (25) : : : : : : : +- CometHashAggregate (24) : : : : : : : +- CometExchange (23) : : : : : : : +- CometHashAggregate (22) @@ -32,7 +32,7 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) +: : : : : : +- CometNativeColumnarToRow (44) : : : : : : +- CometHashAggregate (43) : : : : : : +- CometExchange (42) : : : : : : +- CometHashAggregate (41) @@ -52,7 +52,7 @@ : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) +: : : : : +- CometNativeColumnarToRow (65) : : : : : +- CometHashAggregate (64) : : : : : +- CometExchange (63) : : : : : +- CometHashAggregate (62) @@ -72,7 +72,7 @@ : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) +: : : : +- CometNativeColumnarToRow (86) : : : : +- CometHashAggregate (85) : : : : +- CometExchange (84) : : : : +- CometHashAggregate (83) @@ -92,7 +92,7 @@ : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) +: : : +- CometNativeColumnarToRow (107) : : : +- CometHashAggregate (106) : : : +- CometExchange (105) : : : +- CometHashAggregate (104) @@ -112,7 +112,7 @@ : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) +: : +- CometNativeColumnarToRow (128) : : +- CometHashAggregate (127) : : +- CometExchange (126) : : +- CometHashAggregate (125) @@ -132,7 +132,7 @@ : : : +- CometNativeScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) +: +- CometNativeColumnarToRow (149) : +- CometHashAggregate (148) : +- CometExchange (147) : +- CometHashAggregate (146) @@ -152,7 +152,7 @@ : : +- CometNativeScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * CometColumnarToRow (170) + +- CometNativeColumnarToRow (170) +- CometHashAggregate (169) +- CometExchange (168) +- CometHashAggregate (167) @@ -285,7 +285,7 @@ Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -(25) CometColumnarToRow [codegen id : 8] +(25) CometNativeColumnarToRow Input [1]: [h8_30_to_9#14] (26) CometNativeScan parquet spark_catalog.default.store_sales @@ -368,14 +368,14 @@ Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [h9_to_9_30#25] (45) BroadcastExchange Input [1]: [h9_to_9_30#25] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 8] +(46) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -459,14 +459,14 @@ Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -(65) CometColumnarToRow [codegen id : 2] +(65) CometNativeColumnarToRow Input [1]: [h9_30_to_10#36] (66) BroadcastExchange Input [1]: [h9_30_to_10#36] Arguments: IdentityBroadcastMode, [plan_id=5] -(67) BroadcastNestedLoopJoin [codegen id : 8] +(67) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -550,14 +550,14 @@ Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -(86) CometColumnarToRow [codegen id : 3] +(86) CometNativeColumnarToRow Input [1]: [h10_to_10_30#47] (87) BroadcastExchange Input [1]: [h10_to_10_30#47] Arguments: IdentityBroadcastMode, [plan_id=7] -(88) BroadcastNestedLoopJoin [codegen id : 8] +(88) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -641,14 +641,14 @@ Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -(107) CometColumnarToRow [codegen id : 4] +(107) CometNativeColumnarToRow Input [1]: [h10_30_to_11#58] (108) BroadcastExchange Input [1]: [h10_30_to_11#58] Arguments: IdentityBroadcastMode, [plan_id=9] -(109) BroadcastNestedLoopJoin [codegen id : 8] +(109) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -732,14 +732,14 @@ Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -(128) CometColumnarToRow [codegen id : 5] +(128) CometNativeColumnarToRow Input [1]: [h11_to_11_30#69] (129) BroadcastExchange Input [1]: [h11_to_11_30#69] Arguments: IdentityBroadcastMode, [plan_id=11] -(130) BroadcastNestedLoopJoin [codegen id : 8] +(130) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -823,14 +823,14 @@ Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -(149) CometColumnarToRow [codegen id : 6] +(149) CometNativeColumnarToRow Input [1]: [h11_30_to_12#80] (150) BroadcastExchange Input [1]: [h11_30_to_12#80] Arguments: IdentityBroadcastMode, [plan_id=13] -(151) BroadcastNestedLoopJoin [codegen id : 8] +(151) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -914,14 +914,14 @@ Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -(170) CometColumnarToRow [codegen id : 7] +(170) CometNativeColumnarToRow Input [1]: [h12_to_12_30#91] (171) BroadcastExchange Input [1]: [h12_to_12_30#91] Arguments: IdentityBroadcastMode, [plan_id=15] -(172) BroadcastNestedLoopJoin [codegen id : 8] +(172) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/extended.txt index 24f596de9b..87f19deeb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/extended.txt @@ -5,7 +5,7 @@ BroadcastNestedLoopJoin : : : :- BroadcastNestedLoopJoin : : : : :- BroadcastNestedLoopJoin : : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : : : :- CometColumnarToRow +: : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometHashAggregate : : : : : : : +- CometExchange : : : : : : : +- CometHashAggregate @@ -31,7 +31,7 @@ BroadcastNestedLoopJoin : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : : : +- BroadcastExchange -: : : : : : +- CometColumnarToRow +: : : : : : +- CometNativeColumnarToRow : : : : : : +- CometHashAggregate : : : : : : +- CometExchange : : : : : : +- CometHashAggregate @@ -57,7 +57,7 @@ BroadcastNestedLoopJoin : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : : +- BroadcastExchange -: : : : : +- CometColumnarToRow +: : : : : +- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometExchange : : : : : +- CometHashAggregate @@ -83,7 +83,7 @@ BroadcastNestedLoopJoin : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : +- BroadcastExchange -: : : : +- CometColumnarToRow +: : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -109,7 +109,7 @@ BroadcastNestedLoopJoin : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange -: : : +- CometColumnarToRow +: : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -135,7 +135,7 @@ BroadcastNestedLoopJoin : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -161,7 +161,7 @@ BroadcastNestedLoopJoin : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -187,7 +187,7 @@ BroadcastNestedLoopJoin : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/simplified.txt index cef8e0e760..82809b3066 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (8) +WholeStageCodegen (1) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin @@ -6,8 +6,8 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [count] [h8_30_to_9,count(1)] CometExchange #1 CometHashAggregate [count] @@ -34,57 +34,9 @@ WholeStageCodegen (8) CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 + CometNativeColumnarToRow + CometHashAggregate [count] [h9_to_9_30,count(1)] + CometExchange #6 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -96,18 +48,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #13 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 + InputAdapter + BroadcastExchange #8 + CometNativeColumnarToRow + CometHashAggregate [count] [h9_30_to_10,count(1)] + CometExchange #9 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -119,18 +69,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #16 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 + InputAdapter + BroadcastExchange #11 + CometNativeColumnarToRow + CometHashAggregate [count] [h10_to_10_30,count(1)] + CometExchange #12 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -142,18 +90,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #19 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 + InputAdapter + BroadcastExchange #14 + CometNativeColumnarToRow + CometHashAggregate [count] [h10_30_to_11,count(1)] + CometExchange #15 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -165,18 +111,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #22 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 + InputAdapter + BroadcastExchange #17 + CometNativeColumnarToRow + CometHashAggregate [count] [h11_to_11_30,count(1)] + CometExchange #18 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -188,8 +132,50 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #25 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #20 + CometNativeColumnarToRow + CometHashAggregate [count] [h11_30_to_12,count(1)] + CometExchange #21 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #22 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #23 + CometNativeColumnarToRow + CometHashAggregate [count] [h12_to_12_30,count(1)] + CometExchange #24 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #25 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/explain.txt index 56583c6d26..2a3e4162b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (30) +- * Project (29) +- * Filter (28) +- Window (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -16,7 +16,7 @@ TakeOrderedAndProject (30) : +- * BroadcastHashJoin Inner BuildRight (12) : :- * Project (10) : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject (30) : : +- Scan parquet spark_catalog.default.store_sales (5) : +- ReusedExchange (11) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -46,7 +46,7 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) AS i_class#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) AS i_category#7] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] (5) Scan parquet spark_catalog.default.store_sales @@ -68,26 +68,26 @@ Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] (11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#13, d_moy#14] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#11] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14] Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#14] @@ -102,24 +102,24 @@ ReadSchema: struct Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] Condition : isnotnull(s_store_sk#15) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] (17) BroadcastExchange Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#9] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14, s_store_sk#15, s_store_name#16, s_company_name#17] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] @@ -130,10 +130,10 @@ Results [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_nam Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] @@ -148,18 +148,18 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#16, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#16 ASC NULLS FIRST, s_company_name#17 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] (27) Window Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#16, s_company_name#17, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#16, s_company_name#17] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END -(29) Project [codegen id : 7] +(29) Project [codegen id : 4] Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] @@ -171,7 +171,7 @@ Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -192,7 +192,7 @@ Condition : ((isnotnull(d_year#24) AND (d_year#24 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#13, d_year#24, d_moy#14] Arguments: [d_date_sk#13, d_moy#14], [d_date_sk#13, d_moy#14] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_moy#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt index 0b6c2edaa0..5c7a26057d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject +- Project +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -25,17 +25,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/simplified.txt index f87ef33db4..573b3eeaf0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/simplified.txt @@ -1,53 +1,47 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk,s_store_name,s_company_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/explain.txt index 47ce0d4a91..801d3151d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_moy#15] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/simplified.txt index 61bfd1d960..e64a4daac6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt index 47ce0d4a91..801d3151d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_moy#15] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 61bfd1d960..e64a4daac6 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/explain.txt index da194f2825..927b31b896 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * CometColumnarToRow (3) ++- CometNativeColumnarToRow (3) +- CometFilter (2) +- CometNativeScan parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) CometColumnarToRow [codegen id : 1] +(3) CometNativeColumnarToRow Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] @@ -26,7 +26,7 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) +CometNativeColumnarToRow (12) +- CometProject (11) +- CometHashAggregate (10) +- CometExchange (9) @@ -69,7 +69,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(Unscal Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] -(12) CometColumnarToRow [codegen id : 1] +(12) CometNativeColumnarToRow Input [1]: [mergedValue#29] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] @@ -77,7 +77,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) +CometNativeColumnarToRow (20) +- CometProject (19) +- CometHashAggregate (18) +- CometExchange (17) @@ -120,7 +120,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(Unscal Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] -(20) CometColumnarToRow [codegen id : 1] +(20) CometNativeColumnarToRow Input [1]: [mergedValue#42] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] @@ -128,7 +128,7 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) +CometNativeColumnarToRow (28) +- CometProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -171,7 +171,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(Unscal Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [mergedValue#55] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] @@ -179,7 +179,7 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) +CometNativeColumnarToRow (36) +- CometProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -222,7 +222,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(Unscal Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [mergedValue#68] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] @@ -230,7 +230,7 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) +CometNativeColumnarToRow (44) +- CometProject (43) +- CometHashAggregate (42) +- CometExchange (41) @@ -273,7 +273,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(Unscal Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [mergedValue#81] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/extended.txt index 53ba3252cf..ff39852ed0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/extended.txt @@ -1,6 +1,6 @@ Project [COMET: ] : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -11,7 +11,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -22,7 +22,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -33,7 +33,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -44,7 +44,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -54,7 +54,7 @@ : : +- CometNativeScan parquet spark_catalog.default.store_sales : :- ReusedSubquery : +- ReusedSubquery -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/simplified.txt index 9593d6ebd1..813cb91b2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/simplified.txt @@ -1,71 +1,61 @@ WholeStageCodegen (1) Project Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [r_reason_sk] CometNativeScan parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/explain.txt index b3f32555de..14dc1ef68d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * CometColumnarToRow (3) ++- CometNativeColumnarToRow (3) +- CometFilter (2) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) CometColumnarToRow [codegen id : 1] +(3) CometNativeColumnarToRow Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt index ed71033b26..a86d85c820 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/extended.txt @@ -54,7 +54,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : :- ReusedSubquery : +- ReusedSubquery -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/simplified.txt index 558f5f4b36..9b54130ef1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/simplified.txt @@ -65,7 +65,7 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [r_reason_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt index b3f32555de..14dc1ef68d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * CometColumnarToRow (3) ++- CometNativeColumnarToRow (3) +- CometFilter (2) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) CometColumnarToRow [codegen id : 1] +(3) CometNativeColumnarToRow Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/extended.txt index ed71033b26..a86d85c820 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/extended.txt @@ -54,7 +54,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : :- ReusedSubquery : +- ReusedSubquery -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason 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 558f5f4b36..9b54130ef1 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 @@ -65,7 +65,7 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [r_reason_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/explain.txt index 73b54f439b..71001884ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) + :- CometNativeColumnarToRow (25) : +- CometHashAggregate (24) : +- CometExchange (23) : +- CometHashAggregate (22) @@ -27,7 +27,7 @@ : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometHashAggregate (43) +- CometExchange (42) +- CometHashAggregate (41) @@ -160,7 +160,7 @@ Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -(25) CometColumnarToRow [codegen id : 2] +(25) CometNativeColumnarToRow Input [1]: [amc#12] (26) CometNativeScan parquet spark_catalog.default.web_sales @@ -243,18 +243,18 @@ Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [pmc#22] (45) BroadcastExchange Input [1]: [pmc#22] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 2] +(46) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(47) Project [codegen id : 2] +(47) Project [codegen id : 1] Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] Input [2]: [amc#12, pmc#22] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/extended.txt index 977fcbbc6b..b1125e15e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/extended.txt @@ -1,6 +1,6 @@ Project +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -26,7 +26,7 @@ Project : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/simplified.txt index a13072cd91..686e05404c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (2) +WholeStageCodegen (1) Project [amc,pmc] BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [count] [amc,count(1)] CometExchange #1 CometHashAggregate [count] @@ -29,24 +29,22 @@ WholeStageCodegen (2) CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - ReusedExchange [wp_web_page_sk] #4 + CometNativeColumnarToRow + CometHashAggregate [count] [pmc,count(1)] + CometExchange #6 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #7 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/explain.txt index b52840adb9..9ae2954307 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -18,7 +18,7 @@ : : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : : :- * Project (10) : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * CometColumnarToRow (4) + : : : : : :- CometNativeColumnarToRow (4) : : : : : : +- CometProject (3) : : : : : : +- CometFilter (2) : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center (1) @@ -28,21 +28,21 @@ : : : : : +- Scan parquet spark_catalog.default.catalog_returns (5) : : : : +- ReusedExchange (11) : : : +- BroadcastExchange (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometFilter (15) : : : +- CometNativeScan parquet spark_catalog.default.customer (14) : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometProject (22) : : +- CometFilter (21) : : +- CometNativeScan parquet spark_catalog.default.customer_address (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_demographics (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.household_demographics (34) @@ -63,7 +63,7 @@ Condition : isnotnull(cc_call_center_sk#1) Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cc_call_center_id#2, 16, true, false, true) AS cc_call_center_id#5, cc_name#3, cc_manager#4] -(4) CometColumnarToRow [codegen id : 7] +(4) CometNativeColumnarToRow Input [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] (5) Scan parquet spark_catalog.default.catalog_returns @@ -85,26 +85,26 @@ Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_ Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cc_call_center_sk#1] Right keys [1]: [cr_call_center_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 2] Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] (11) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#11] -(12) BroadcastHashJoin [codegen id : 7] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returned_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 7] +(13) Project [codegen id : 2] Output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] @@ -119,20 +119,20 @@ ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 3] Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) @@ -62,20 +62,20 @@ Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotn Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [i_item_sk#5] (8) BroadcastExchange Input [1]: [i_item_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 6] +(9) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 6] +(10) Project [codegen id : 3] Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] @@ -87,27 +87,27 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] +(12) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -(13) Filter [codegen id : 3] +(13) Filter [codegen id : 1] Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] Condition : isnotnull(ws_item_sk#7) (14) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#10] -(15) BroadcastHashJoin [codegen id : 3] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(16) Project [codegen id : 3] +(16) Project [codegen id : 1] Output [2]: [ws_item_sk#7, ws_ext_discount_amt#8] Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#10] -(17) HashAggregate [codegen id : 3] +(17) HashAggregate [codegen id : 1] Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] Keys [1]: [ws_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] @@ -118,17 +118,17 @@ Results [3]: [ws_item_sk#7, sum#13, count#14] Input [3]: [ws_item_sk#7, sum#13, count#14] Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) CometColumnarToRow [codegen id : 4] +(19) CometNativeColumnarToRow Input [3]: [ws_item_sk#7, sum#13, count#14] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#7, sum#13, count#14] Keys [1]: [ws_item_sk#7] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#15] Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -(21) Filter [codegen id : 4] +(21) Filter [codegen id : 2] Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) @@ -136,30 +136,30 @@ Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#5] Right keys [1]: [ws_item_sk#7] Join type: Inner Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#16) -(24) Project [codegen id : 6] +(24) Project [codegen id : 3] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] (25) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#17] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 3] Output [1]: [ws_ext_discount_amt#2] Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#17] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 3] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] @@ -170,10 +170,10 @@ Results [1]: [sum#19] Input [1]: [sum#19] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [1]: [sum#19] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 4] Input [1]: [sum#19] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] @@ -184,7 +184,7 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#20,17,2) AS Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -205,7 +205,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#2 Input [2]: [d_date_sk#17, d_date#22] Arguments: [d_date_sk#17], [d_date_sk#17] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt index 2d15266049..b3ca85cdaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -13,19 +13,19 @@ HashAggregate : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -35,12 +35,12 @@ HashAggregate : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/simplified.txt index 17df728966..71aeded80b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [ws_ext_discount_amt] [sum,sum] Project [ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -17,29 +17,25 @@ WholeStageCodegen (7) Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(ws_ext_discount_amt))] HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ws_item_sk] #5 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] Project [ws_item_sk,ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/explain.txt index cdc8dbc3bc..1b2f594125 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/explain.txt @@ -175,7 +175,7 @@ Input [1]: [Excess Discount Amount #18] Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt index 20df9a8047..ca50f78e18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/simplified.txt index 4ec5755df7..991100027b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/simplified.txt @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt index cdc8dbc3bc..1b2f594125 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt @@ -175,7 +175,7 @@ Input [1]: [Excess Discount Amount #18] Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/extended.txt index 20df9a8047..ca50f78e18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 4ec5755df7..991100027b 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 @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/explain.txt index 3f38adcd39..6a067b40ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (22) +CometNativeColumnarToRow (22) +- CometTakeOrderedAndProject (21) +- CometHashAggregate (20) +- CometExchange (19) @@ -119,6 +119,6 @@ Functions [1]: [sum(act_sales#14)] Input [2]: [ss_customer_sk#2, sumsales#17] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] -(22) CometColumnarToRow [codegen id : 1] +(22) CometNativeColumnarToRow Input [2]: [ss_customer_sk#2, sumsales#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/extended.txt index 7203a678b0..1496f26233 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/simplified.txt index c0b74010df..f08009ad54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/simplified.txt @@ -1,24 +1,22 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - 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] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] +CometNativeColumnarToRow + CometTakeOrderedAndProject [ss_customer_sk,sumsales] + CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] + CometExchange [ss_customer_sk] #1 + 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] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometExchange [ss_item_sk,ss_ticket_number] #2 + CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometExchange [sr_item_sk,sr_ticket_number] #3 + CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometBroadcastExchange [r_reason_sk] #4 + CometProject [r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] + CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/explain.txt index 1caeb8ca76..b09a5a035c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (41) +CometNativeColumnarToRow (41) +- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -204,7 +204,7 @@ Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [3]: [ws_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -230,6 +230,6 @@ Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -(41) CometColumnarToRow [codegen id : 2] +(41) CometNativeColumnarToRow Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/extended.txt index e5c7b8688a..c91f0410ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/extended.txt @@ -1,9 +1,9 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/simplified.txt index feab73bcd6..cd992a9669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/simplified.txt @@ -1,45 +1,43 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_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] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_warehouse_sk] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] +CometNativeColumnarToRow + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + 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] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_warehouse_sk] + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometExchange [ws_order_number] #2 + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometSort [ws_warehouse_sk,ws_order_number] + CometExchange [ws_order_number] #3 + CometProject [ws_warehouse_sk,ws_order_number] + CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometSort [wr_order_number] + CometExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/explain.txt index 39f45f7a74..867ac7a785 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (54) +CometNativeColumnarToRow (54) +- CometHashAggregate (53) +- CometColumnarExchange (52) +- * HashAggregate (51) +- * HashAggregate (50) - +- * CometColumnarToRow (49) + +- CometNativeColumnarToRow (49) +- CometHashAggregate (48) +- CometProject (47) +- CometBroadcastHashJoin (46) @@ -271,7 +271,7 @@ Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [3]: [ws_order_number#4, sum#25, sum#26] (50) HashAggregate [codegen id : 1] @@ -297,6 +297,6 @@ Input [3]: [sum#25, sum#26, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -(54) CometColumnarToRow [codegen id : 2] +(54) CometNativeColumnarToRow Input [3]: [order count #31, total shipping cost #32, total net profit #33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/extended.txt index cbba03d28d..94e8b8d087 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/extended.txt @@ -1,9 +1,9 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/simplified.txt index 15f29c507c..b81cc72852 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/simplified.txt @@ -1,58 +1,56 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_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] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] +CometNativeColumnarToRow + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + 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] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometExchange [ws_order_number] #2 + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometProject [ws_order_number] + CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] + CometSort [ws_warehouse_sk,ws_order_number] + CometExchange [ws_order_number] #3 + CometProject [ws_warehouse_sk,ws_order_number] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometProject [wr_order_number] + CometSortMergeJoin [wr_order_number,ws_order_number] + CometSort [wr_order_number] + CometExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + CometProject [ws_order_number] + CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/explain.txt index db701aae34..7ca69ec813 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (25) +CometNativeColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -138,6 +138,6 @@ Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -(25) CometColumnarToRow [codegen id : 1] +(25) CometNativeColumnarToRow Input [1]: [count(1)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/extended.txt index 170d3b7879..9ee165830a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/simplified.txt index b40f41659f..6599ccccb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/simplified.txt @@ -1,27 +1,25 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] +CometNativeColumnarToRow + CometHashAggregate [count] [count(1),count(1)] + CometExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/explain.txt index a7f1cb451d..0121fd183f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (24) +CometNativeColumnarToRow (24) +- CometHashAggregate (23) +- CometExchange (22) +- CometHashAggregate (21) @@ -32,23 +32,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] (3) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] -(4) BroadcastHashJoin [codegen id : 2] +(4) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(5) Project [codegen id : 2] +(5) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_customer_sk#2] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] -(6) HashAggregate [codegen id : 2] +(6) HashAggregate [codegen id : 1] Input [2]: [ss_item_sk#1, ss_customer_sk#2] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] @@ -75,23 +75,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 2] Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] (12) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#11] -(13) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#10] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(14) Project [codegen id : 4] +(14) Project [codegen id : 2] Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#11] -(15) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 2] Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Functions: [] @@ -134,14 +134,14 @@ Input [3]: [sum#14, sum#15, sum#16] Keys: [] Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END)] -(24) CometColumnarToRow [codegen id : 5] +(24) CometNativeColumnarToRow Input [3]: [store_only#17, catalog_only#18, store_and_catalog#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -162,7 +162,7 @@ Condition : (((isnotnull(d_month_seq#20) AND (d_month_seq#20 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#20] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt index 798bfdaf1b..e3b777a05a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -14,12 +14,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -33,7 +33,7 @@ CometColumnarToRow : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/simplified.txt index dac440b122..04cb3e457d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/simplified.txt @@ -1,41 +1,37 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - 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 [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] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (4) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 +CometNativeColumnarToRow + 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 [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] + CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] + CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_customer_sk,ss_item_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + CometSort [customer_sk,item_sk] + CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] + CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 + WholeStageCodegen (2) + HashAggregate [cs_bill_customer_sk,cs_item_sk] + Project [cs_bill_customer_sk,cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/explain.txt index 420f37fccc..a21fa2b802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/explain.txt @@ -146,7 +146,7 @@ Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt index e1aefb8382..0969a0e796 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/simplified.txt index a7de047b25..d37700ec0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/simplified.txt @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt index 420f37fccc..a21fa2b802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt @@ -146,7 +146,7 @@ Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/extended.txt index e1aefb8382..0969a0e796 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a7de047b25..d37700ec0f 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 @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/explain.txt index 8263680b2b..dcc1ecf5c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -* CometColumnarToRow (26) +CometNativeColumnarToRow (26) +- CometProject (25) +- CometSort (24) +- CometColumnarExchange (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -20,7 +20,7 @@ : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.store_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -35,10 +35,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) @@ -57,37 +57,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -98,10 +98,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -116,14 +116,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] (21) Window Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] @@ -139,14 +139,14 @@ Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrev Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(26) CometColumnarToRow [codegen id : 7] +(26) CometNativeColumnarToRow Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.date_dim (27) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt index 030031856f..af8cf29205 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt @@ -1,14 +1,14 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometProject +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -20,17 +20,17 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/simplified.txt index 3f64b57ffa..91f3f3c423 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/simplified.txt @@ -1,48 +1,40 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] +CometNativeColumnarToRow + CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + CometNativeColumnarToRow + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/explain.txt index 4cc725ef2b..8ef0f76fe1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/explain.txt @@ -147,7 +147,7 @@ Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrev Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt index 4c972848e7..152665febf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/simplified.txt index af05a7d8c6..8c213757be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/simplified.txt @@ -24,12 +24,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt index 4cc725ef2b..8ef0f76fe1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt @@ -147,7 +147,7 @@ Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrev Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt index 4c972848e7..152665febf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 af05a7d8c6..8c213757be 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 @@ -24,12 +24,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/explain.txt index 3447b6efa3..0035984c4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (29) +CometNativeColumnarToRow (29) +- CometTakeOrderedAndProject (28) +- CometHashAggregate (27) +- CometExchange (26) @@ -163,6 +163,6 @@ Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt index c075163849..c236a1647f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/simplified.txt index 98a215dbc6..4c6dcfd6fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/simplified.txt @@ -1,31 +1,29 @@ -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 [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 [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] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +CometNativeColumnarToRow + 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 [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 [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] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometProject [sm_type] [sm_ship_mode_sk,sm_type] + CometFilter [sm_ship_mode_sk,sm_type] + CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/explain.txt index d1964eee2a..c1e71a36fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (44) : : +- * BroadcastHashJoin Inner BuildRight (28) : : :- * Filter (11) : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometColumnarExchange (8) : : : +- * HashAggregate (7) : : : +- * Project (6) @@ -20,11 +20,11 @@ TakeOrderedAndProject (44) : : +- BroadcastExchange (27) : : +- * Filter (26) : : +- * HashAggregate (25) - : : +- * CometColumnarToRow (24) + : : +- CometNativeColumnarToRow (24) : : +- CometColumnarExchange (23) : : +- * HashAggregate (22) : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (20) + : : +- CometNativeColumnarToRow (20) : : +- CometColumnarExchange (19) : : +- * HashAggregate (18) : : +- * Project (17) @@ -34,12 +34,12 @@ TakeOrderedAndProject (44) : : : +- Scan parquet spark_catalog.default.store_returns (12) : : +- ReusedExchange (15) : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- CometProject (32) : +- CometFilter (31) : +- CometNativeScan parquet spark_catalog.default.store (30) +- BroadcastExchange (41) - +- * CometColumnarToRow (40) + +- CometNativeColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometNativeScan parquet spark_catalog.default.customer (37) @@ -53,27 +53,27 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 2] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [sr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 2] +(6) Project [codegen id : 1] Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -(7) HashAggregate [codegen id : 2] +(7) HashAggregate [codegen id : 1] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] @@ -84,17 +84,17 @@ Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 9] +(9) CometNativeColumnarToRow Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -(10) HashAggregate [codegen id : 9] +(10) HashAggregate [codegen id : 5] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(11) Filter [codegen id : 9] +(11) Filter [codegen id : 5] Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] Condition : isnotnull(ctr_total_return#12) @@ -106,27 +106,27 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 2] Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -(14) Filter [codegen id : 4] +(14) Filter [codegen id : 2] Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] Condition : isnotnull(sr_store_sk#14) (15) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#17] -(16) BroadcastHashJoin [codegen id : 4] +(16) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#16] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(17) Project [codegen id : 4] +(17) Project [codegen id : 2] Output [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#17] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] Keys [2]: [sr_customer_sk#13, sr_store_sk#14] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))] @@ -137,17 +137,17 @@ Results [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -(21) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 3] Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] Keys [2]: [sr_customer_sk#13, sr_store_sk#14] Functions [1]: [sum(UnscaledValue(sr_return_amt#15))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9] Results [2]: [sr_store_sk#14 AS ctr_store_sk#20, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#21] -(22) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 3] Input [2]: [ctr_store_sk#20, ctr_total_return#21] Keys [1]: [ctr_store_sk#20] Functions [1]: [partial_avg(ctr_total_return#21)] @@ -158,17 +158,17 @@ Results [3]: [ctr_store_sk#20, sum#24, count#25] Input [3]: [ctr_store_sk#20, sum#24, count#25] Arguments: hashpartitioning(ctr_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 6] +(24) CometNativeColumnarToRow Input [3]: [ctr_store_sk#20, sum#24, count#25] -(25) HashAggregate [codegen id : 6] +(25) HashAggregate [codegen id : 4] Input [3]: [ctr_store_sk#20, sum#24, count#25] Keys [1]: [ctr_store_sk#20] Functions [1]: [avg(ctr_total_return#21)] Aggregate Attributes [1]: [avg(ctr_total_return#21)#26] Results [2]: [(avg(ctr_total_return#21)#26 * 1.2) AS (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -(26) Filter [codegen id : 6] +(26) Filter [codegen id : 4] Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) @@ -176,13 +176,13 @@ Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] -(28) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_store_sk#11] Right keys [1]: [ctr_store_sk#20] Join type: Inner Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27) -(29) Project [codegen id : 9] +(29) Project [codegen id : 5] Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] @@ -201,20 +201,20 @@ Condition : ((isnotnull(s_state#29) AND (static_invoke(CharVarcharCodegenUtils.r Input [2]: [s_store_sk#28, s_state#29] Arguments: [s_store_sk#28], [s_store_sk#28] -(33) CometColumnarToRow [codegen id : 7] +(33) CometNativeColumnarToRow Input [1]: [s_store_sk#28] (34) BroadcastExchange Input [1]: [s_store_sk#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(35) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_store_sk#11] Right keys [1]: [s_store_sk#28] Join type: Inner Join condition: None -(36) Project [codegen id : 9] +(36) Project [codegen id : 5] Output [1]: [ctr_customer_sk#10] Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#28] @@ -233,20 +233,20 @@ Condition : isnotnull(c_customer_sk#30) Input [2]: [c_customer_sk#30, c_customer_id#31] Arguments: [c_customer_sk#30, c_customer_id#32], [c_customer_sk#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#31, 16)) AS c_customer_id#32] -(40) CometColumnarToRow [codegen id : 8] +(40) CometNativeColumnarToRow Input [2]: [c_customer_sk#30, c_customer_id#32] (41) BroadcastExchange Input [2]: [c_customer_sk#30, c_customer_id#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(42) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_customer_sk#10] Right keys [1]: [c_customer_sk#30] Join type: Inner Join condition: None -(43) Project [codegen id : 9] +(43) Project [codegen id : 5] Output [1]: [c_customer_id#32] Input [3]: [ctr_customer_sk#10, c_customer_sk#30, c_customer_id#32] @@ -258,7 +258,7 @@ Arguments: 100, [c_customer_id#32 ASC NULLS FIRST], [c_customer_id#32] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometNativeScan parquet spark_catalog.default.date_dim (45) @@ -279,7 +279,7 @@ Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#6, d_year#33] Arguments: [d_date_sk#6], [d_date_sk#6] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt index 0622aad9d5..91b07a7538 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -17,23 +17,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -43,17 +43,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/simplified.txt index ed85c142aa..1351d8c439 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (9) + WholeStageCodegen (5) Project [c_customer_id] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk] @@ -8,10 +8,10 @@ TakeOrderedAndProject [c_customer_id] BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] Project [sr_customer_sk,sr_store_sk,sr_return_amt] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -21,29 +21,27 @@ TakeOrderedAndProject [c_customer_id] Scan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (6) + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ctr_store_sk] #4 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] Project [sr_customer_sk,sr_store_sk,sr_return_amt] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -56,17 +54,13 @@ TakeOrderedAndProject [c_customer_id] ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] + CometNativeColumnarToRow + CometProject [c_customer_id] [c_customer_sk,c_customer_id] + CometFilter [c_customer_sk,c_customer_id] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt index a37054da2d..c9193adb11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt @@ -236,7 +236,7 @@ Input [1]: [c_customer_id#28] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt index 8aa14c43dd..0d0d80c7f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt index 3e1d6243f8..d69c52dda0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt index a37054da2d..c9193adb11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt @@ -236,7 +236,7 @@ Input [1]: [c_customer_id#28] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt index 8aa14c43dd..0d0d80c7f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 3e1d6243f8..d69c52dda0 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/explain.txt index 2c4b495826..2ea1cdcad0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -13,7 +13,7 @@ TakeOrderedAndProject (45) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -35,12 +35,12 @@ TakeOrderedAndProject (45) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_address (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) @@ -57,7 +57,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] (6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] @@ -87,7 +87,7 @@ Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi @@ -100,19 +100,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] (13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] @@ -120,7 +120,7 @@ Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) @@ -133,19 +133,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] @@ -153,17 +153,17 @@ Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(25) Filter [codegen id : 4] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -182,20 +182,20 @@ Condition : (ca_county#17 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#16, ca_county#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [1]: [ca_address_sk#16] (31) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16] @@ -214,24 +214,24 @@ Condition : isnotnull(cd_demo_sk#18) Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#19, 1)) AS cd_gender#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#29, cd_purchase_estimate#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#23, 10)) AS cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] (38) BroadcastExchange Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 4] Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 4] Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [partial_count(1)] @@ -242,10 +242,10 @@ Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_pur Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 5] Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [count(1)] @@ -260,7 +260,7 @@ Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FI Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * CometColumnarToRow (49) ++- CometNativeColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometNativeScan parquet spark_catalog.default.date_dim (46) @@ -281,7 +281,7 @@ Condition : (((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = Input [3]: [d_date_sk#9, d_year#40, d_moy#41] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt index 8935b6cc13..40a8ef4a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -22,12 +22,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,17 +49,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/simplified.txt index 0cc108b0b0..ff26eb68a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -29,17 +29,15 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,7 +48,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -61,17 +59,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/explain.txt index e253b8ca7b..0686a1af42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [1]: [ca_address_sk#20] (33) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#22) Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#23, 1)) AS cd_gender#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#24, 1)) AS cd_marital_status#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#25, 20)) AS cd_education_status#33, cd_purchase_estimate#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#27, 10)) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (40) BroadcastExchange Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] @@ -253,10 +253,10 @@ Results [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_pur Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] @@ -271,7 +271,7 @@ Arguments: 100, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/simplified.txt index e7193f87e1..fd519f33e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt index e253b8ca7b..0686a1af42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [1]: [ca_address_sk#20] (33) BroadcastExchange Input [1]: [ca_address_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#22) Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#23, 1)) AS cd_gender#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#24, 1)) AS cd_marital_status#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#25, 20)) AS cd_education_status#33, cd_purchase_estimate#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#27, 10)) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (40) BroadcastExchange Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#22] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] @@ -253,10 +253,10 @@ Results [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_pur Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] @@ -271,7 +271,7 @@ Arguments: 100, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt index e7193f87e1..fd519f33e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/explain.txt index 0b7aa0c883..26615c3dae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/explain.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject (80) : : +- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -26,14 +26,14 @@ TakeOrderedAndProject (80) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -45,14 +45,14 @@ TakeOrderedAndProject (80) : +- BroadcastExchange (57) : +- * Filter (56) : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) + : +- CometNativeColumnarToRow (54) : +- CometColumnarExchange (53) : +- * HashAggregate (52) : +- * Project (51) : +- * BroadcastHashJoin Inner BuildRight (50) : :- * Project (48) : : +- * BroadcastHashJoin Inner BuildRight (47) - : : :- * CometColumnarToRow (42) + : : :- CometNativeColumnarToRow (42) : : : +- CometProject (41) : : : +- CometFilter (40) : : : +- CometNativeScan parquet spark_catalog.default.customer (39) @@ -63,14 +63,14 @@ TakeOrderedAndProject (80) : +- ReusedExchange (49) +- BroadcastExchange (77) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- CometColumnarExchange (74) +- * HashAggregate (73) +- * Project (72) +- * BroadcastHashJoin Inner BuildRight (71) :- * Project (69) : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * CometColumnarToRow (63) + : :- CometNativeColumnarToRow (63) : : +- CometProject (62) : : +- CometFilter (61) : : +- CometNativeScan parquet spark_catalog.default.customer (60) @@ -96,7 +96,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharC Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] (5) Scan parquet spark_catalog.default.store_sales @@ -118,30 +118,30 @@ Condition : isnotnull(ss_customer_sk#15) Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#15] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] (11) ReusedExchange [Reuses operator id: 84] Output [2]: [d_date_sk#20, d_year#21] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] @@ -152,17 +152,17 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#25, year_total#26] Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) @@ -181,7 +181,7 @@ Condition : (isnotnull(c_customer_sk#27) AND isnotnull(static_invoke(CharVarchar Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] Arguments: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40], [c_customer_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#28, 16)) AS c_customer_id#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#29, 20)) AS c_first_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#30, 30)) AS c_last_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#31, 1)) AS c_preferred_cust_flag#38, c_birth_country#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#33, 13)) AS c_login#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#34, 50)) AS c_email_address#40] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [8]: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] (23) Scan parquet spark_catalog.default.store_sales @@ -192,10 +192,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] Condition : isnotnull(ss_customer_sk#41) @@ -203,30 +203,30 @@ Condition : isnotnull(ss_customer_sk#41) Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#27] Right keys [1]: [ss_customer_sk#41] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] Input [12]: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] (29) ReusedExchange [Reuses operator id: 88] Output [2]: [d_date_sk#46, d_year#47] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#44] Right keys [1]: [d_date_sk#46] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, d_year#47] Input [12]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, d_year#47] Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))] @@ -237,10 +237,10 @@ Results [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_pr Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] Arguments: hashpartitioning(c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))] @@ -251,13 +251,13 @@ Results [3]: [c_customer_id#35 AS customer_id#50, c_preferred_cust_flag#38 AS cu Input [3]: [customer_id#50, customer_preferred_cust_flag#51, year_total#52] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#50] Join type: Inner Join condition: None -(38) Project [codegen id : 16] +(38) Project [codegen id : 12] Output [4]: [customer_id#25, year_total#26, customer_preferred_cust_flag#51, year_total#52] Input [5]: [customer_id#25, year_total#26, customer_id#50, customer_preferred_cust_flag#51, year_total#52] @@ -276,7 +276,7 @@ Condition : (isnotnull(c_customer_sk#53) AND isnotnull(static_invoke(CharVarchar Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] Arguments: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66], [c_customer_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)) AS c_customer_id#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#55, 20)) AS c_first_name#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#56, 30)) AS c_last_name#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#57, 1)) AS c_preferred_cust_flag#64, c_birth_country#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#59, 13)) AS c_login#65, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#60, 50)) AS c_email_address#66] -(42) CometColumnarToRow [codegen id : 10] +(42) CometNativeColumnarToRow Input [8]: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66] (43) Scan parquet spark_catalog.default.web_sales @@ -287,10 +287,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#70), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 8] +(44) ColumnarToRow [codegen id : 6] Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -(45) Filter [codegen id : 8] +(45) Filter [codegen id : 6] Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] Condition : isnotnull(ws_bill_customer_sk#67) @@ -298,30 +298,30 @@ Condition : isnotnull(ws_bill_customer_sk#67) Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#53] Right keys [1]: [ws_bill_customer_sk#67] Join type: Inner Join condition: None -(48) Project [codegen id : 10] +(48) Project [codegen id : 7] Output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] Input [12]: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] (49) ReusedExchange [Reuses operator id: 84] Output [2]: [d_date_sk#71, d_year#72] -(50) BroadcastHashJoin [codegen id : 10] +(50) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#70] Right keys [1]: [d_date_sk#71] Join type: Inner Join condition: None -(51) Project [codegen id : 10] +(51) Project [codegen id : 7] Output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#72] Input [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70, d_date_sk#71, d_year#72] -(52) HashAggregate [codegen id : 10] +(52) HashAggregate [codegen id : 7] Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#72] Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] @@ -332,17 +332,17 @@ Results [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cus Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72, sum#74] Arguments: hashpartitioning(c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometColumnarToRow [codegen id : 11] +(54) CometNativeColumnarToRow Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72, sum#74] -(55) HashAggregate [codegen id : 11] +(55) HashAggregate [codegen id : 8] Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72, sum#74] Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))#75] Results [2]: [c_customer_id#61 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))#75,18,2) AS year_total#77] -(56) Filter [codegen id : 11] +(56) Filter [codegen id : 8] Input [2]: [customer_id#76, year_total#77] Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.00)) @@ -350,13 +350,13 @@ Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.00)) Input [2]: [customer_id#76, year_total#77] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 16] +(58) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#76] Join type: Inner Join condition: None -(59) Project [codegen id : 16] +(59) Project [codegen id : 12] Output [5]: [customer_id#25, year_total#26, customer_preferred_cust_flag#51, year_total#52, year_total#77] Input [6]: [customer_id#25, year_total#26, customer_preferred_cust_flag#51, year_total#52, customer_id#76, year_total#77] @@ -375,7 +375,7 @@ Condition : (isnotnull(c_customer_sk#78) AND isnotnull(static_invoke(CharVarchar Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] Arguments: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91], [c_customer_sk#78, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#79, 16)) AS c_customer_id#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#80, 20)) AS c_first_name#87, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#81, 30)) AS c_last_name#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#82, 1)) AS c_preferred_cust_flag#89, c_birth_country#83, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#84, 13)) AS c_login#90, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#85, 50)) AS c_email_address#91] -(63) CometColumnarToRow [codegen id : 14] +(63) CometNativeColumnarToRow Input [8]: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91] (64) Scan parquet spark_catalog.default.web_sales @@ -386,10 +386,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 12] +(65) ColumnarToRow [codegen id : 9] Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -(66) Filter [codegen id : 12] +(66) Filter [codegen id : 9] Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] Condition : isnotnull(ws_bill_customer_sk#92) @@ -397,30 +397,30 @@ Condition : isnotnull(ws_bill_customer_sk#92) Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(68) BroadcastHashJoin [codegen id : 14] +(68) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#78] Right keys [1]: [ws_bill_customer_sk#92] Join type: Inner Join condition: None -(69) Project [codegen id : 14] +(69) Project [codegen id : 10] Output [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] Input [12]: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] (70) ReusedExchange [Reuses operator id: 88] Output [2]: [d_date_sk#96, d_year#97] -(71) BroadcastHashJoin [codegen id : 14] +(71) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#95] Right keys [1]: [d_date_sk#96] Join type: Inner Join condition: None -(72) Project [codegen id : 14] +(72) Project [codegen id : 10] Output [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#97] Input [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95, d_date_sk#96, d_year#97] -(73) HashAggregate [codegen id : 14] +(73) HashAggregate [codegen id : 10] Input [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#97] Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))] @@ -431,10 +431,10 @@ Results [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cus Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97, sum#99] Arguments: hashpartitioning(c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(75) CometColumnarToRow [codegen id : 15] +(75) CometNativeColumnarToRow Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97, sum#99] -(76) HashAggregate [codegen id : 15] +(76) HashAggregate [codegen id : 11] Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97, sum#99] Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))] @@ -445,13 +445,13 @@ Results [2]: [c_customer_id#86 AS customer_id#100, MakeDecimal(sum(UnscaledValue Input [2]: [customer_id#100, year_total#101] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(78) BroadcastHashJoin [codegen id : 16] +(78) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#100] Join type: Inner Join condition: (CASE WHEN (year_total#77 > 0.00) THEN (year_total#101 / year_total#77) END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#52 / year_total#26) END) -(79) Project [codegen id : 16] +(79) Project [codegen id : 12] Output [1]: [customer_preferred_cust_flag#51] Input [7]: [customer_id#25, year_total#26, customer_preferred_cust_flag#51, year_total#52, year_total#77, customer_id#100, year_total#101] @@ -463,7 +463,7 @@ Arguments: 100, [customer_preferred_cust_flag#51 ASC NULLS FIRST], [customer_pre Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (84) -+- * CometColumnarToRow (83) ++- CometNativeColumnarToRow (83) +- CometFilter (82) +- CometNativeScan parquet spark_catalog.default.date_dim (81) @@ -479,7 +479,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(83) CometColumnarToRow [codegen id : 1] +(83) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (84) BroadcastExchange @@ -488,7 +488,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#45 BroadcastExchange (88) -+- * CometColumnarToRow (87) ++- CometNativeColumnarToRow (87) +- CometFilter (86) +- CometNativeScan parquet spark_catalog.default.date_dim (85) @@ -504,7 +504,7 @@ ReadSchema: struct Input [2]: [d_date_sk#46, d_year#47] Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#46)) -(87) CometColumnarToRow [codegen id : 1] +(87) CometNativeColumnarToRow Input [2]: [d_date_sk#46, d_year#47] (88) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt index 0f9f19de77..47aa9517cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -24,23 +24,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -50,24 +50,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -77,19 +77,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -99,7 +99,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/simplified.txt index 6c9e276c01..80b29cda63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] @@ -8,17 +8,17 @@ TakeOrderedAndProject [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] @@ -31,69 +31,65 @@ TakeOrderedAndProject [customer_preferred_cust_flag] Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -103,25 +99,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt index b6edfa8800..78dbf9d442 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt @@ -420,7 +420,7 @@ Input [1]: [customer_preferred_cust_flag#48] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometFilter (75) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (77) BroadcastExchange @@ -445,7 +445,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometFilter (79) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) @@ -461,7 +461,7 @@ ReadSchema: struct Input [2]: [d_date_sk#44, d_year#45] Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#44, d_year#45] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt index 781eae9054..63858a8376 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -44,7 +44,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt index 4599ecf539..0cc605f69e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt @@ -24,11 +24,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -48,11 +46,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt index b6edfa8800..78dbf9d442 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt @@ -420,7 +420,7 @@ Input [1]: [customer_preferred_cust_flag#48] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometFilter (75) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (77) BroadcastExchange @@ -445,7 +445,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometFilter (79) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) @@ -461,7 +461,7 @@ ReadSchema: struct Input [2]: [d_date_sk#44, d_year#45] Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#44, d_year#45] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt index 781eae9054..63858a8376 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -44,7 +44,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 4599ecf539..0cc605f69e 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 @@ -24,11 +24,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -48,11 +46,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/explain.txt index 409078c363..8b8521a9c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.web_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) @@ -54,37 +54,37 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10 Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] (21) Window Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt index 6c2a775097..15648408fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/simplified.txt index b7b0a89774..b580349644 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt index 8811d04e59..b48a1033bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt index fb83fd2f9a..c014d569f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt index 8811d04e59..b48a1033bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fb83fd2f9a..c014d569f4 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/explain.txt index 613d0cb7b2..59599686de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * HashAggregate (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometColumnarExchange (34) +- * HashAggregate (33) +- * Project (32) @@ -17,22 +17,22 @@ : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) + : : : : +- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.store (4) : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometProject (12) : : : +- CometFilter (11) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) : : +- ReusedExchange (17) : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometProject (22) : +- CometFilter (21) : +- CometNativeScan parquet spark_catalog.default.customer_demographics (20) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.household_demographics (27) @@ -45,10 +45,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 1] Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 1] Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) @@ -63,20 +63,20 @@ ReadSchema: struct Input [1]: [s_store_sk#12] Condition : isnotnull(s_store_sk#12) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [s_store_sk#12] (7) BroadcastExchange Input [1]: [s_store_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 1] Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] @@ -95,33 +95,33 @@ Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) AS ca_state#16] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [2]: [ca_address_sk#13, ca_state#16] (14) BroadcastExchange Input [2]: [ca_address_sk#13, ca_state#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#3] Right keys [1]: [ca_address_sk#13] Join type: Inner Join condition: ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) -(16) Project [codegen id : 6] +(16) Project [codegen id : 1] Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] (17) ReusedExchange [Reuses operator id: 41] Output [1]: [d_date_sk#17] -(18) BroadcastHashJoin [codegen id : 6] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#10] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(19) Project [codegen id : 1] Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] @@ -140,20 +140,20 @@ Condition : (isnotnull(cd_demo_sk#18) AND ((((static_invoke(CharVarcharCodegenUt Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] Arguments: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22], [cd_demo_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) AS cd_marital_status#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#20, 20)) AS cd_education_status#22] -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] (24) BroadcastExchange Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: ((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) -(26) Project [codegen id : 6] +(26) Project [codegen id : 1] Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22] Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] @@ -168,24 +168,24 @@ ReadSchema: struct Input [2]: [hd_demo_sk#23, hd_dep_count#24] Condition : (isnotnull(hd_demo_sk#23) AND ((hd_dep_count#24 = 3) OR (hd_dep_count#24 = 1))) -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [2]: [hd_demo_sk#23, hd_dep_count#24] (30) BroadcastExchange Input [2]: [hd_demo_sk#23, hd_dep_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#23] Join type: Inner Join condition: (((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#24 = 3)) OR (((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#24 = 1))) OR (((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#24 = 1))) -(32) Project [codegen id : 6] +(32) Project [codegen id : 1] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22, hd_demo_sk#23, hd_dep_count#24] -(33) HashAggregate [codegen id : 6] +(33) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Keys: [] Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] @@ -196,10 +196,10 @@ Results [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] -(36) HashAggregate [codegen id : 7] +(36) HashAggregate [codegen id : 2] Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] @@ -210,7 +210,7 @@ Results [4]: [avg(ss_quantity#5)#39 AS avg(ss_quantity)#43, cast((avg(UnscaledVa Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (41) -+- * CometColumnarToRow (40) ++- CometNativeColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometNativeScan parquet spark_catalog.default.date_dim (37) @@ -231,7 +231,7 @@ Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#17, d_year#47] Arguments: [d_date_sk#17], [d_date_sk#17] -(40) CometColumnarToRow [codegen id : 1] +(40) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (41) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt index 4c0d0b7a33..9ff51aab85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,31 +17,31 @@ HashAggregate : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/simplified.txt index a33ae5a161..d7dc065e83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) +WholeStageCodegen (2) HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (6) + WholeStageCodegen (1) HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] @@ -21,41 +21,31 @@ WholeStageCodegen (7) Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometNativeColumnarToRow + CometFilter [hd_demo_sk,hd_dep_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt index 027d679419..5c8bee106e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt @@ -200,7 +200,7 @@ Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesal Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -221,7 +221,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#17, d_year#18] Arguments: [d_date_sk#17], [d_date_sk#17] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt index 08e9beb692..5568227543 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt index 8ef882a435..3d2ac96c74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt index 027d679419..5c8bee106e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt @@ -200,7 +200,7 @@ Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesal Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -221,7 +221,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#17, d_year#18] Arguments: [d_date_sk#17], [d_date_sk#17] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt index 08e9beb692..5568227543 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 8ef882a435..3d2ac96c74 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/explain.txt index f7377ccf8a..ad732bc935 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (110) +- * HashAggregate (109) - +- * CometColumnarToRow (108) + +- CometNativeColumnarToRow (108) +- CometColumnarExchange (107) +- * HashAggregate (106) +- * Expand (105) @@ -9,7 +9,7 @@ TakeOrderedAndProject (110) :- * Project (69) : +- * Filter (68) : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) + : +- CometNativeColumnarToRow (66) : +- CometColumnarExchange (65) : +- * HashAggregate (64) : +- * Project (63) @@ -23,12 +23,12 @@ TakeOrderedAndProject (110) : : : +- BroadcastExchange (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) + : : : :- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (48) : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) + : : : :- CometNativeColumnarToRow (36) : : : : +- CometHashAggregate (35) : : : : +- CometColumnarExchange (34) : : : : +- * HashAggregate (33) @@ -41,7 +41,7 @@ TakeOrderedAndProject (110) : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : +- BroadcastExchange (27) : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) + : : : : : :- CometNativeColumnarToRow (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- BroadcastExchange (25) @@ -53,7 +53,7 @@ TakeOrderedAndProject (110) : : : : : : : +- * ColumnarToRow (14) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometNativeColumnarToRow (18) : : : : : : +- CometFilter (17) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : +- ReusedExchange (22) @@ -70,7 +70,7 @@ TakeOrderedAndProject (110) : : : +- ReusedExchange (43) : : +- BroadcastExchange (58) : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) + : : :- CometNativeColumnarToRow (55) : : : +- CometFilter (54) : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : +- ReusedExchange (56) @@ -78,7 +78,7 @@ TakeOrderedAndProject (110) :- * Project (86) : +- * Filter (85) : +- * HashAggregate (84) - : +- * CometColumnarToRow (83) + : +- CometNativeColumnarToRow (83) : +- CometColumnarExchange (82) : +- * HashAggregate (81) : +- * Project (80) @@ -95,7 +95,7 @@ TakeOrderedAndProject (110) +- * Project (103) +- * Filter (102) +- * HashAggregate (101) - +- * CometColumnarToRow (100) + +- CometNativeColumnarToRow (100) +- CometColumnarExchange (99) +- * HashAggregate (98) +- * Project (97) @@ -119,10 +119,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -137,7 +137,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -166,7 +166,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -195,33 +195,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -326,7 +326,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -336,13 +336,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -350,7 +350,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -367,13 +367,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(69) Project [codegen id : 26] +(69) Project [codegen id : 15] Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#37 AS i_brand_id#54, i_class_id#38 AS i_class_id#55, i_category_id#39 AS i_category_id#56] Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] @@ -443,17 +443,17 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 51] +(71) ColumnarToRow [codegen id : 29] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -(72) Filter [codegen id : 51] +(72) Filter [codegen id : 29] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Condition : isnotnull(cs_item_sk#57) (73) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#61] -(74) BroadcastHashJoin [codegen id : 51] +(74) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#57] Right keys [1]: [ss_item_sk#61] Join type: LeftSemi @@ -462,30 +462,30 @@ Join condition: None (75) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -(76) BroadcastHashJoin [codegen id : 51] +(76) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#57] Right keys [1]: [i_item_sk#62] Join type: Inner Join condition: None -(77) Project [codegen id : 51] +(77) Project [codegen id : 29] Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] (78) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#66] -(79) BroadcastHashJoin [codegen id : 51] +(79) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(80) Project [codegen id : 51] +(80) Project [codegen id : 29] Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] -(81) HashAggregate [codegen id : 51] +(81) HashAggregate [codegen id : 29] Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] @@ -496,21 +496,21 @@ Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71 Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(83) CometColumnarToRow [codegen id : 52] +(83) CometNativeColumnarToRow Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -(84) HashAggregate [codegen id : 52] +(84) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73, count(1)#74] Results [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73 AS sales#75, count(1)#74 AS number_sales#76] -(85) Filter [codegen id : 52] +(85) Filter [codegen id : 30] Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#75, number_sales#76] Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(86) Project [codegen id : 52] +(86) Project [codegen id : 30] Output [6]: [sales#75, number_sales#76, catalog AS channel#77, i_brand_id#63, i_class_id#64, i_category_id#65] Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#75, number_sales#76] @@ -522,17 +522,17 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(88) ColumnarToRow [codegen id : 77] +(88) ColumnarToRow [codegen id : 44] Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -(89) Filter [codegen id : 77] +(89) Filter [codegen id : 44] Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] Condition : isnotnull(ws_item_sk#78) (90) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#82] -(91) BroadcastHashJoin [codegen id : 77] +(91) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#78] Right keys [1]: [ss_item_sk#82] Join type: LeftSemi @@ -541,30 +541,30 @@ Join condition: None (92) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] -(93) BroadcastHashJoin [codegen id : 77] +(93) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#78] Right keys [1]: [i_item_sk#83] Join type: Inner Join condition: None -(94) Project [codegen id : 77] +(94) Project [codegen id : 44] Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86] Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] (95) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#87] -(96) BroadcastHashJoin [codegen id : 77] +(96) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#81] Right keys [1]: [d_date_sk#87] Join type: Inner Join condition: None -(97) Project [codegen id : 77] +(97) Project [codegen id : 44] Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86, d_date_sk#87] -(98) HashAggregate [codegen id : 77] +(98) HashAggregate [codegen id : 44] Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] @@ -575,31 +575,31 @@ Results [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92 Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] Arguments: hashpartitioning(i_brand_id#84, i_class_id#85, i_category_id#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(100) CometColumnarToRow [codegen id : 78] +(100) CometNativeColumnarToRow Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] -(101) HashAggregate [codegen id : 78] +(101) HashAggregate [codegen id : 45] Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94, count(1)#95] Results [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94 AS sales#96, count(1)#95 AS number_sales#97] -(102) Filter [codegen id : 78] +(102) Filter [codegen id : 45] Input [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sales#96, number_sales#97] Condition : (isnotnull(sales#96) AND (cast(sales#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(103) Project [codegen id : 78] +(103) Project [codegen id : 45] Output [6]: [sales#96, number_sales#97, web AS channel#98, i_brand_id#84, i_class_id#85, i_category_id#86] Input [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sales#96, number_sales#97] (104) Union -(105) Expand [codegen id : 79] +(105) Expand [codegen id : 46] Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] -(106) HashAggregate [codegen id : 79] +(106) HashAggregate [codegen id : 46] Input [7]: [sales#49, number_sales#50, channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] Keys [5]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] @@ -610,10 +610,10 @@ Results [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spa Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] Arguments: hashpartitioning(channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(108) CometColumnarToRow [codegen id : 80] +(108) CometNativeColumnarToRow Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] -(109) HashAggregate [codegen id : 80] +(109) HashAggregate [codegen id : 47] Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] Keys [5]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] Functions [2]: [sum(sales#49), sum(number_sales#50)] @@ -628,7 +628,7 @@ Arguments: 100, [channel#99 ASC NULLS FIRST, i_brand_id#100 ASC NULLS FIRST, i_c Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * HashAggregate (130) -+- * CometColumnarToRow (129) ++- CometNativeColumnarToRow (129) +- CometColumnarExchange (128) +- * HashAggregate (127) +- Union (126) @@ -656,19 +656,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#116), dynamicpruningexpression(ss_sold_date_sk#116 IN dynamicpruning#12)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 2] +(112) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116] (113) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#117] -(114) BroadcastHashJoin [codegen id : 2] +(114) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#116] Right keys [1]: [d_date_sk#117] Join type: Inner Join condition: None -(115) Project [codegen id : 2] +(115) Project [codegen id : 1] Output [2]: [ss_quantity#114 AS quantity#118, ss_list_price#115 AS list_price#119] Input [4]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116, d_date_sk#117] @@ -679,19 +679,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#122), dynamicpruningexpression(cs_sold_date_sk#122 IN dynamicpruning#12)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 4] +(117) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122] (118) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#123] -(119) BroadcastHashJoin [codegen id : 4] +(119) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#122] Right keys [1]: [d_date_sk#123] Join type: Inner Join condition: None -(120) Project [codegen id : 4] +(120) Project [codegen id : 2] Output [2]: [cs_quantity#120 AS quantity#124, cs_list_price#121 AS list_price#125] Input [4]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122, d_date_sk#123] @@ -702,25 +702,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_sold_date_sk#128 IN dynamicpruning#12)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 6] +(122) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128] (123) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#129] -(124) BroadcastHashJoin [codegen id : 6] +(124) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#128] Right keys [1]: [d_date_sk#129] Join type: Inner Join condition: None -(125) Project [codegen id : 6] +(125) Project [codegen id : 3] Output [2]: [ws_quantity#126 AS quantity#130, ws_list_price#127 AS list_price#131] Input [4]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128, d_date_sk#129] (126) Union -(127) HashAggregate [codegen id : 7] +(127) HashAggregate [codegen id : 4] Input [2]: [quantity#118, list_price#119] Keys: [] Functions [1]: [partial_avg((cast(quantity#118 as decimal(10,0)) * list_price#119))] @@ -731,10 +731,10 @@ Results [2]: [sum#134, count#135] Input [2]: [sum#134, count#135] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(129) CometColumnarToRow [codegen id : 8] +(129) CometNativeColumnarToRow Input [2]: [sum#134, count#135] -(130) HashAggregate [codegen id : 8] +(130) HashAggregate [codegen id : 5] Input [2]: [sum#134, count#135] Keys: [] Functions [1]: [avg((cast(quantity#118 as decimal(10,0)) * list_price#119))] @@ -749,7 +749,7 @@ Subquery:4 Hosting operator id = 121 Hosting Expression = ws_sold_date_sk#128 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (135) -+- * CometColumnarToRow (134) ++- CometNativeColumnarToRow (134) +- CometProject (133) +- CometFilter (132) +- CometNativeScan parquet spark_catalog.default.date_dim (131) @@ -770,7 +770,7 @@ Condition : ((((isnotnull(d_year#138) AND isnotnull(d_moy#139)) AND (d_year#138 Input [3]: [d_date_sk#40, d_year#138, d_moy#139] Arguments: [d_date_sk#40], [d_date_sk#40] -(134) CometColumnarToRow [codegen id : 1] +(134) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (135) BroadcastExchange @@ -779,7 +779,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (140) -+- * CometColumnarToRow (139) ++- CometNativeColumnarToRow (139) +- CometProject (138) +- CometFilter (137) +- CometNativeScan parquet spark_catalog.default.date_dim (136) @@ -800,7 +800,7 @@ Condition : (((isnotnull(d_year#140) AND (d_year#140 >= 1999)) AND (d_year#140 < Input [2]: [d_date_sk#24, d_year#140] Arguments: [d_date_sk#24], [d_date_sk#24] -(139) CometColumnarToRow [codegen id : 1] +(139) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (140) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt index dfd3434d90..223593e3d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -9,7 +9,7 @@ TakeOrderedAndProject : +- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,12 +39,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -57,19 +57,19 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -82,13 +82,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -101,16 +101,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -124,28 +124,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -158,13 +158,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -177,16 +177,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -200,16 +200,16 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -217,7 +217,7 @@ TakeOrderedAndProject : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -232,12 +232,12 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -250,13 +250,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -269,16 +269,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -292,28 +292,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -326,13 +326,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -345,16 +345,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -368,16 +368,16 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -385,7 +385,7 @@ TakeOrderedAndProject +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -400,12 +400,12 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -418,13 +418,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -437,16 +437,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -460,28 +460,28 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -494,13 +494,13 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -513,16 +513,16 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -536,16 +536,16 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/simplified.txt index c989fe9a81..f348396ac0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/simplified.txt @@ -1,28 +1,28 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) + WholeStageCodegen (47) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) + WholeStageCodegen (46) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (26) + WholeStageCodegen (15) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #3 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #13 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -31,7 +31,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,10 +50,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [d_date_sk] #7 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -66,30 +66,28 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (11) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (10) + WholeStageCodegen (5) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [brand_id,class_id,category_id] CometColumnarExchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -101,23 +99,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) + WholeStageCodegen (2) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (1) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -129,18 +125,16 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 - WholeStageCodegen (9) + WholeStageCodegen (4) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -156,25 +150,25 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #12 - WholeStageCodegen (23) + WholeStageCodegen (13) BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) + WholeStageCodegen (30) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) + WholeStageCodegen (29) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -192,15 +186,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) + WholeStageCodegen (45) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/explain.txt index 22f1896b63..b77116db1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/explain.txt @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * CometColumnarToRow (128) ++- CometNativeColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 < Input [2]: [d_date_sk#26, d_year#125] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) CometColumnarToRow [codegen id : 1] +(128) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (129) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt index 4af04a7846..49ad7403ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -213,7 +213,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -277,7 +277,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -355,7 +355,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -419,7 +419,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/simplified.txt index 79c782f2ca..fea1950918 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/simplified.txt @@ -44,12 +44,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #4 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -68,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt index 22f1896b63..b77116db1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * CometColumnarToRow (128) ++- CometNativeColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 < Input [2]: [d_date_sk#26, d_year#125] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) CometColumnarToRow [codegen id : 1] +(128) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (129) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/extended.txt index 4af04a7846..49ad7403ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -213,7 +213,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -277,7 +277,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -355,7 +355,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -419,7 +419,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 79c782f2ca..fea1950918 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 @@ -44,12 +44,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #4 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -68,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/explain.txt index 7630396f0b..71732c4f26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (87) +- * BroadcastHashJoin Inner BuildRight (86) :- * Filter (68) : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) + : +- CometNativeColumnarToRow (66) : +- CometColumnarExchange (65) : +- * HashAggregate (64) : +- * Project (63) @@ -17,12 +17,12 @@ TakeOrderedAndProject (87) : : : +- BroadcastExchange (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) + : : : :- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (48) : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) + : : : :- CometNativeColumnarToRow (36) : : : : +- CometHashAggregate (35) : : : : +- CometColumnarExchange (34) : : : : +- * HashAggregate (33) @@ -35,7 +35,7 @@ TakeOrderedAndProject (87) : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : +- BroadcastExchange (27) : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) + : : : : : :- CometNativeColumnarToRow (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- BroadcastExchange (25) @@ -47,7 +47,7 @@ TakeOrderedAndProject (87) : : : : : : : +- * ColumnarToRow (14) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometNativeColumnarToRow (18) : : : : : : +- CometFilter (17) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : +- ReusedExchange (22) @@ -64,7 +64,7 @@ TakeOrderedAndProject (87) : : : +- ReusedExchange (43) : : +- BroadcastExchange (58) : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) + : : :- CometNativeColumnarToRow (55) : : : +- CometFilter (54) : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : +- ReusedExchange (56) @@ -72,7 +72,7 @@ TakeOrderedAndProject (87) +- BroadcastExchange (85) +- * Filter (84) +- * HashAggregate (83) - +- * CometColumnarToRow (82) + +- CometNativeColumnarToRow (82) +- CometColumnarExchange (81) +- * HashAggregate (80) +- * Project (79) @@ -96,10 +96,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -114,7 +114,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -143,7 +143,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -172,33 +172,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -303,7 +303,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -313,13 +313,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -327,7 +327,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -344,13 +344,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -416,17 +416,17 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 50] +(70) ColumnarToRow [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(71) Filter [codegen id : 50] +(71) Filter [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) (72) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#59] -(73) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#59] Join type: LeftSemi @@ -435,30 +435,30 @@ Join condition: None (74) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(75) BroadcastHashJoin [codegen id : 50] +(75) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#60] Join type: Inner Join condition: None -(76) Project [codegen id : 50] +(76) Project [codegen id : 28] Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] (77) ReusedExchange [Reuses operator id: 126] Output [1]: [d_date_sk#64] -(78) BroadcastHashJoin [codegen id : 50] +(78) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(79) Project [codegen id : 50] +(79) Project [codegen id : 28] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -(80) HashAggregate [codegen id : 50] +(80) HashAggregate [codegen id : 28] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] @@ -469,17 +469,17 @@ Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(82) CometColumnarToRow [codegen id : 51] +(82) CometNativeColumnarToRow Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -(83) HashAggregate [codegen id : 51] +(83) HashAggregate [codegen id : 29] Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71, count(1)#72] Results [6]: [store AS channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71 AS sales#74, count(1)#72 AS number_sales#75] -(84) Filter [codegen id : 51] +(84) Filter [codegen id : 29] Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -487,7 +487,7 @@ Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(Reu Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 52] +(86) BroadcastHashJoin [codegen id : 30] Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Join type: Inner @@ -501,7 +501,7 @@ Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * HashAggregate (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometColumnarExchange (105) +- * HashAggregate (104) +- Union (103) @@ -529,19 +529,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#12)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 2] +(89) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] (90) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#79] -(91) BroadcastHashJoin [codegen id : 2] +(91) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#78] Right keys [1]: [d_date_sk#79] Join type: Inner Join condition: None -(92) Project [codegen id : 2] +(92) Project [codegen id : 1] Output [2]: [ss_quantity#76 AS quantity#80, ss_list_price#77 AS list_price#81] Input [4]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#79] @@ -552,19 +552,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#12)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 4] +(94) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] (95) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#85] -(96) BroadcastHashJoin [codegen id : 4] +(96) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#84] Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(97) Project [codegen id : 4] +(97) Project [codegen id : 2] Output [2]: [cs_quantity#82 AS quantity#86, cs_list_price#83 AS list_price#87] Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#85] @@ -575,25 +575,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#12)] ReadSchema: struct -(99) ColumnarToRow [codegen id : 6] +(99) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] (100) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#91] -(101) BroadcastHashJoin [codegen id : 6] +(101) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#90] Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(102) Project [codegen id : 6] +(102) Project [codegen id : 3] Output [2]: [ws_quantity#88 AS quantity#92, ws_list_price#89 AS list_price#93] Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#91] (103) Union -(104) HashAggregate [codegen id : 7] +(104) HashAggregate [codegen id : 4] Input [2]: [quantity#80, list_price#81] Keys: [] Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] @@ -604,10 +604,10 @@ Results [2]: [sum#96, count#97] Input [2]: [sum#96, count#97] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(106) CometColumnarToRow [codegen id : 8] +(106) CometNativeColumnarToRow Input [2]: [sum#96, count#97] -(107) HashAggregate [codegen id : 8] +(107) HashAggregate [codegen id : 5] Input [2]: [sum#96, count#97] Keys: [] Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] @@ -622,7 +622,7 @@ Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (112) -+- * CometColumnarToRow (111) ++- CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometNativeScan parquet spark_catalog.default.date_dim (108) @@ -643,7 +643,7 @@ Condition : ((isnotnull(d_week_seq#100) AND (d_week_seq#100 = ReusedSubquery Sub Input [2]: [d_date_sk#40, d_week_seq#100] Arguments: [d_date_sk#40], [d_date_sk#40] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (112) BroadcastExchange @@ -653,7 +653,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 109 Hosting Expression = ReusedSubquery Subquery scalar-subquery#101, [id=#102] Subquery:7 Hosting operator id = 108 Hosting Expression = Subquery scalar-subquery#101, [id=#102] -* CometColumnarToRow (116) +CometNativeColumnarToRow (116) +- CometProject (115) +- CometFilter (114) +- CometNativeScan parquet spark_catalog.default.date_dim (113) @@ -674,12 +674,12 @@ Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] Arguments: [d_week_seq#103], [d_week_seq#103] -(116) CometColumnarToRow [codegen id : 1] +(116) CometNativeColumnarToRow Input [1]: [d_week_seq#103] Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometNativeScan parquet spark_catalog.default.date_dim (117) @@ -700,7 +700,7 @@ Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1999)) AND (d_year#107 < Input [2]: [d_date_sk#24, d_year#107] Arguments: [d_date_sk#24], [d_date_sk#24] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (121) BroadcastExchange @@ -715,7 +715,7 @@ Subquery:11 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 BroadcastExchange (126) -+- * CometColumnarToRow (125) ++- CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometNativeScan parquet spark_catalog.default.date_dim (122) @@ -736,7 +736,7 @@ Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = ReusedSubquery Sub Input [2]: [d_date_sk#64, d_week_seq#108] Arguments: [d_date_sk#64], [d_date_sk#64] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_date_sk#64] (126) BroadcastExchange @@ -746,7 +746,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:13 Hosting operator id = 123 Hosting Expression = ReusedSubquery Subquery scalar-subquery#109, [id=#110] Subquery:14 Hosting operator id = 122 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* CometColumnarToRow (130) +CometNativeColumnarToRow (130) +- CometProject (129) +- CometFilter (128) +- CometNativeScan parquet spark_catalog.default.date_dim (127) @@ -767,7 +767,7 @@ Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] Arguments: [d_week_seq#111], [d_week_seq#111] -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [1]: [d_week_seq#111] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt index e44d85bdc2..25e8321ad7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject :- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -23,7 +23,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -33,12 +33,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -51,25 +51,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -82,13 +82,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -101,16 +101,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -124,28 +124,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -158,13 +158,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -177,16 +177,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -200,22 +200,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -223,7 +223,7 @@ TakeOrderedAndProject +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -236,25 +236,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -267,13 +267,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -286,16 +286,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -309,28 +309,28 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -343,13 +343,13 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -362,16 +362,16 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -385,22 +385,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim +- Subquery - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/simplified.txt index cc02a716ea..27d519ebbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (30) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #12 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -21,7 +21,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #6 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -56,38 +56,34 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (5) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [brand_id,class_id,category_id] CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -99,23 +95,21 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 - WholeStageCodegen (4) + WholeStageCodegen (2) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #8 - WholeStageCodegen (3) + WholeStageCodegen (1) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -127,18 +121,16 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 - WholeStageCodegen (9) + WholeStageCodegen (4) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -154,10 +146,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 - WholeStageCodegen (23) + WholeStageCodegen (13) BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter @@ -166,14 +158,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (51) + WholeStageCodegen (29) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + WholeStageCodegen (28) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -186,20 +178,16 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/explain.txt index ccc69246c8..f7abbacbf4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subqu Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -623,7 +623,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] Subquery:7 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -644,12 +644,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#99] Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -670,7 +670,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -689,7 +689,7 @@ Subquery:13 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:14 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -710,7 +710,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subqu Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -720,7 +720,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:15 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] Subquery:16 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -741,7 +741,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#104] Subquery:17 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt index b32dd95954..49eafd1015 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt @@ -44,13 +44,13 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,7 +72,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -136,7 +136,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -199,13 +199,13 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -227,7 +227,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -291,7 +291,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/simplified.txt index 6b5264b76d..0d84ade8e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/simplified.txt @@ -39,20 +39,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -71,12 +67,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -133,20 +127,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt index ccc69246c8..f7abbacbf4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subqu Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -623,7 +623,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] Subquery:7 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -644,12 +644,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#99] Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -670,7 +670,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -689,7 +689,7 @@ Subquery:13 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:14 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -710,7 +710,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subqu Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -720,7 +720,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:15 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] Subquery:16 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -741,7 +741,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#104] Subquery:17 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/extended.txt index b32dd95954..49eafd1015 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/extended.txt @@ -44,13 +44,13 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,7 +72,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -136,7 +136,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -199,13 +199,13 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -227,7 +227,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -291,7 +291,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6b5264b76d..0d84ade8e8 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 @@ -39,20 +39,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -71,12 +67,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -133,20 +127,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/explain.txt index ddc9c5e253..6e82737b76 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -14,11 +14,11 @@ TakeOrderedAndProject (24) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) + : : +- CometNativeColumnarToRow (6) : : +- CometFilter (5) : : +- CometNativeScan parquet spark_catalog.default.customer (4) : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) + : +- CometNativeColumnarToRow (13) : +- CometProject (12) : +- CometFilter (11) : +- CometNativeScan parquet spark_catalog.default.customer_address (10) @@ -33,10 +33,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_bill_customer_sk#1) @@ -51,20 +51,20 @@ ReadSchema: struct Input [2]: [c_customer_sk#5, c_current_addr_sk#6] Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [2]: [c_customer_sk#5, c_current_addr_sk#6] (7) BroadcastExchange Input [2]: [c_customer_sk#5, c_current_addr_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#5] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] @@ -83,37 +83,37 @@ Condition : isnotnull(ca_address_sk#7) Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)) AS ca_state#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#9, 10)) AS ca_zip#11] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] (14) BroadcastExchange Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#6] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) -(16) Project [codegen id : 4] +(16) Project [codegen id : 1] Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] (17) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#12] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [2]: [cs_sales_price#2, ca_zip#11] Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [2]: [cs_sales_price#2, ca_zip#11] Keys [1]: [ca_zip#11] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] @@ -124,10 +124,10 @@ Results [2]: [ca_zip#11, sum#14] Input [2]: [ca_zip#11, sum#14] Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [2]: [ca_zip#11, sum#14] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [2]: [ca_zip#11, sum#14] Keys [1]: [ca_zip#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] @@ -142,7 +142,7 @@ Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : ((((isnotnull(d_qoy#18) AND isnotnull(d_year#17)) AND (d_qoy#18 = 2) Input [3]: [d_date_sk#12, d_year#17, d_qoy#18] Arguments: [d_date_sk#12], [d_date_sk#12] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#12] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt index 8ed98bc5cb..d0d170db53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -14,21 +14,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/simplified.txt index f31442dcfe..0c2174ad74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_zip] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [ca_zip,cs_sales_price] [sum,sum] Project [cs_sales_price,ca_zip] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -18,26 +18,20 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt index 8a4fb53ece..43724267e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt @@ -142,7 +142,7 @@ Input [2]: [ca_zip#11, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2) Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#12] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt index 6de0c64850..55382845fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt index c39b96efe3..97974aec5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt index 8a4fb53ece..43724267e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt @@ -142,7 +142,7 @@ Input [2]: [ca_zip#11, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2) Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#12] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt index 6de0c64850..55382845fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 c39b96efe3..97974aec5d 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 @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/explain.txt index 2c1be26c56..ad301895b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (41) +CometNativeColumnarToRow (41) +- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -204,7 +204,7 @@ Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [3]: [cs_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -230,6 +230,6 @@ Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -(41) CometColumnarToRow [codegen id : 2] +(41) CometNativeColumnarToRow Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/extended.txt index 9889eeaca3..cb53684dcf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/extended.txt @@ -1,9 +1,9 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/simplified.txt index 429d83d08c..3254fa6c33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/simplified.txt @@ -1,45 +1,43 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_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] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_warehouse_sk] - CometSort [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometExchange [cs_order_number] #2 - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] +CometNativeColumnarToRow + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + 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] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] + CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] + CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_warehouse_sk] + CometSort [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometExchange [cs_order_number] #2 + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + CometSort [cs_warehouse_sk,cs_order_number] + CometExchange [cs_order_number] #3 + CometProject [cs_warehouse_sk,cs_order_number] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + CometSort [cr_order_number] + CometExchange [cr_order_number] #4 + CometProject [cr_order_number] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cc_call_center_sk] #7 + CometProject [cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/explain.txt index f6ba10374e..ea6f82ded6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- ReusedExchange (19) : : +- ReusedExchange (22) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.store (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.item (32) @@ -52,10 +52,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) @@ -78,13 +78,13 @@ Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnot Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 8] +(8) BroadcastHashJoin [codegen id : 3] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] Join type: Inner Join condition: None -(9) Project [codegen id : 8] +(9) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] @@ -107,52 +107,52 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 3] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] Join type: Inner Join condition: None -(15) Project [codegen id : 8] +(15) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] (16) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#18] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(18) Project [codegen id : 3] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#18] (19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#19] -(20) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_returned_date_sk#12] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(21) Project [codegen id : 8] +(21) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] (22) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(24) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] @@ -171,20 +171,20 @@ Condition : isnotnull(s_store_sk#21) Input [2]: [s_store_sk#21, s_state#22] Arguments: [s_store_sk#21, s_state#23], [s_store_sk#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#22, 2)) AS s_state#23] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [2]: [s_store_sk#21, s_state#23] (29) BroadcastExchange Input [2]: [s_store_sk#21, s_state#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#21, s_state#23] @@ -203,24 +203,24 @@ Condition : isnotnull(i_item_sk#24) Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] Arguments: [i_item_sk#24, i_item_id#27, i_item_desc#26], [i_item_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#25, 16)) AS i_item_id#27, i_item_desc#26] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] (36) BroadcastExchange Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#24] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(38) Project [codegen id : 3] Output [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_sk#24, i_item_id#27, i_item_desc#26] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 3] Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] @@ -231,10 +231,10 @@ Results [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] Arguments: hashpartitioning(i_item_id#27, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 4] Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] @@ -249,7 +249,7 @@ Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((isnotnull(d_quarter_name#85) AND (static_invoke(CharVarcharCodegen Input [2]: [d_date_sk#18, d_quarter_name#85] Arguments: [d_date_sk#18], [d_date_sk#18] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#18] (48) BroadcastExchange @@ -279,7 +279,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -300,7 +300,7 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_nam Input [2]: [d_date_sk#19, d_quarter_name#86] Arguments: [d_date_sk#19], [d_date_sk#19] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt index 26e924fd7e..15c6bec5dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,27 +42,27 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/simplified.txt index c63dd716a1..591d904435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,12 +39,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -63,17 +59,13 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk,i_item_id,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/explain.txt index e6d049b949..ab5709b1eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/explain.txt @@ -256,7 +256,7 @@ Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((isnotnull(d_quarter_name#20) AND (static_invoke(CharVarcharCodegen Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_nam Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#21] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/simplified.txt index 1d48f96b09..a274ade9c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/simplified.txt @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt index e6d049b949..ab5709b1eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt @@ -256,7 +256,7 @@ Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((isnotnull(d_quarter_name#20) AND (static_invoke(CharVarcharCodegen Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_nam Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#21] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 1d48f96b09..a274ade9c5 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 @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/explain.txt index f807104667..3068d63ca5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (46) +- * HashAggregate (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometColumnarExchange (43) +- * HashAggregate (42) +- * Expand (41) @@ -21,27 +21,27 @@ TakeOrderedAndProject (46) : : : : : : +- * ColumnarToRow (2) : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : +- BroadcastExchange (8) - : : : : : +- * CometColumnarToRow (7) + : : : : : +- CometNativeColumnarToRow (7) : : : : : +- CometProject (6) : : : : : +- CometFilter (5) : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : : : +- BroadcastExchange (15) - : : : : +- * CometColumnarToRow (14) + : : : : +- CometNativeColumnarToRow (14) : : : : +- CometProject (13) : : : : +- CometFilter (12) : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) : : : +- BroadcastExchange (21) - : : : +- * CometColumnarToRow (20) + : : : +- CometNativeColumnarToRow (20) : : : +- CometFilter (19) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) : : +- BroadcastExchange (28) - : : +- * CometColumnarToRow (27) + : : +- CometNativeColumnarToRow (27) : : +- CometProject (26) : : +- CometFilter (25) : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) : +- ReusedExchange (31) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.item (34) @@ -55,10 +55,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) @@ -77,20 +77,20 @@ Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) A Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [cd_demo_sk#11, cd_dep_count#14] (8) BroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 1] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] @@ -109,20 +109,20 @@ Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15 Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) CometColumnarToRow [codegen id : 2] +(14) CometNativeColumnarToRow Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] (15) BroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 7] +(16) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(17) Project [codegen id : 7] +(17) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] @@ -137,20 +137,20 @@ ReadSchema: struct Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [1]: [cd_demo_sk#20] (21) BroadcastExchange Input [1]: [cd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 1] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -169,33 +169,33 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) AS ca_state#25, ca_country#24] -(27) CometColumnarToRow [codegen id : 4] +(27) CometNativeColumnarToRow Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (28) BroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(30) Project [codegen id : 7] +(30) Project [codegen id : 1] Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (31) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#26] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] @@ -214,28 +214,28 @@ Condition : isnotnull(i_item_sk#27) Input [2]: [i_item_sk#27, i_item_id#28] Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#28, 16)) AS i_item_id#29] -(37) CometColumnarToRow [codegen id : 6] +(37) CometNativeColumnarToRow Input [2]: [i_item_sk#27, i_item_id#29] (38) BroadcastExchange Input [2]: [i_item_sk#27, i_item_id#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 7] +(39) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(40) Project [codegen id : 7] +(40) Project [codegen id : 1] Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] -(41) Expand [codegen id : 7] +(41) Expand [codegen id : 1] Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -(42) HashAggregate [codegen id : 7] +(42) HashAggregate [codegen id : 1] Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] @@ -246,10 +246,10 @@ Results [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_gro Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(44) CometColumnarToRow [codegen id : 8] +(44) CometNativeColumnarToRow Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -(45) HashAggregate [codegen id : 8] +(45) HashAggregate [codegen id : 2] Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] @@ -264,7 +264,7 @@ Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (51) -+- * CometColumnarToRow (50) ++- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometNativeScan parquet spark_catalog.default.date_dim (47) @@ -285,7 +285,7 @@ Condition : ((isnotnull(d_year#77) AND (d_year#77 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#77] Arguments: [d_date_sk#26], [d_date_sk#26] -(50) CometColumnarToRow [codegen id : 1] +(50) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt index b47fce49b3..53fa7b6342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -21,36 +21,36 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/simplified.txt index 77a45c46cd..0162997d1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] @@ -25,50 +25,38 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeColumnarToRow + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeColumnarToRow + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometNativeColumnarToRow + CometFilter [cd_demo_sk] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/explain.txt index 674e83325e..e238aead81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/explain.txt @@ -249,7 +249,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, ag Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt index 0b554c7e7c..7958390823 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/simplified.txt index 6c2b8b2e4f..6abfbb11b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt index 674e83325e..e238aead81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt @@ -249,7 +249,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, ag Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/extended.txt index 0b554c7e7c..7958390823 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6c2b8b2e4f..6abfbb11b2 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 @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/explain.txt index 0e34c62d0c..df585877ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (36) +CometNativeColumnarToRow (36) +- CometTakeOrderedAndProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -203,6 +203,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt index fbe6798e3d..fa37da7c94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/simplified.txt index 675500cd0b..f6eefbf9a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/simplified.txt @@ -1,38 +1,36 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_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 [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] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] - CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] - CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_zip] +CometNativeColumnarToRow + CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_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 [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] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] + CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 + CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_zip] #5 + CometProject [ca_zip] [ca_address_sk,ca_zip] + CometFilter [ca_address_sk,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometProject [s_zip] [s_store_sk,s_zip] + CometFilter [s_store_sk,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/explain.txt index 7ade9d6af8..9d51e70273 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (34) +CometNativeColumnarToRow (34) +- CometSort (33) +- CometExchange (32) +- CometProject (31) @@ -188,6 +188,6 @@ Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREM Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/extended.txt index 7e6f4d3a3c..c3d8dd11af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/simplified.txt index e4b6e81639..6d51b41514 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/simplified.txt @@ -1,36 +1,34 @@ -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)] - 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] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - 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_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] - CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [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] - CometFilter [d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] +CometNativeColumnarToRow + 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)] + 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] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + 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_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] + CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [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] + CometFilter [d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/explain.txt index aa781d42a5..a57467cf02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.catalog_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) @@ -54,37 +54,37 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10 Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] (21) Window Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt index 369ec68bb4..6634edcbed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/simplified.txt index fb0ed62abe..21e87e03b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt index e6612e7574..66f8bdef36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt index 2958d060fe..b3a099fb40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt index e6612e7574..66f8bdef36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 2958d060fe..b3a099fb40 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/explain.txt index fbabcc7275..ecee843bb4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (25) +- * Filter (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -15,11 +15,11 @@ TakeOrderedAndProject (25) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.inventory (1) : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) + : : +- CometNativeColumnarToRow (6) : : +- CometFilter (5) : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) + : +- CometNativeColumnarToRow (13) : +- CometProject (12) : +- CometFilter (11) : +- CometNativeScan parquet spark_catalog.default.item (10) @@ -34,10 +34,10 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) @@ -52,20 +52,20 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Condition : isnotnull(w_warehouse_sk#6) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] (7) BroadcastExchange Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#6] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] @@ -84,37 +84,37 @@ Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) A Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#9, 16)) AS i_item_id#11] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [2]: [i_item_sk#8, i_item_id#11] (14) BroadcastExchange Input [2]: [i_item_sk#8, i_item_id#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(16) Project [codegen id : 4] +(16) Project [codegen id : 1] Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] (17) ReusedExchange [Reuses operator id: 29] Output [2]: [d_date_sk#12, d_date#13] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] Keys [2]: [w_warehouse_name#7, i_item_id#11] Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] @@ -125,17 +125,17 @@ Results [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] Keys [2]: [w_warehouse_name#7, i_item_id#11] Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] Aggregate Attributes [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19] Results [4]: [w_warehouse_name#7, i_item_id#11, sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_before#20, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19 AS inv_after#21] -(24) Filter [codegen id : 5] +(24) Filter [codegen id : 2] Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] Condition : (CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) <= 1.5) END) @@ -147,7 +147,7 @@ Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRS Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_date#13] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt index 51fcfd010a..407648ea84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,20 +15,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/simplified.txt index 94925f8911..7c6d29a502 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) + WholeStageCodegen (2) Filter [inv_before,inv_after] HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] BroadcastHashJoin [inv_date_sk,d_date_sk] @@ -19,25 +19,19 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/explain.txt index 69e7bad682..90c17e30d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/explain.txt @@ -142,7 +142,7 @@ Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_date#13] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt index ef02334507..097f5b20cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/simplified.txt index 1c2e80c991..9fc6fdc42b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/simplified.txt @@ -16,11 +16,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 CometFilter [w_warehouse_sk,w_warehouse_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt index 69e7bad682..90c17e30d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt @@ -142,7 +142,7 @@ Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_date#13] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/extended.txt index ef02334507..097f5b20cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 1c2e80c991..9fc6fdc42b 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 @@ -16,11 +16,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 CometFilter [w_warehouse_sk,w_warehouse_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/explain.txt index 4de342c4d8..e67515a30a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (25) +- * HashAggregate (24) - +- * CometColumnarToRow (23) + +- CometNativeColumnarToRow (23) +- CometColumnarExchange (22) +- * HashAggregate (21) +- * Expand (20) @@ -16,12 +16,12 @@ TakeOrderedAndProject (25) : : : +- Scan parquet spark_catalog.default.inventory (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.warehouse (14) @@ -34,23 +34,23 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (4) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] @@ -69,20 +69,20 @@ Condition : isnotnull(i_item_sk#7) Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#8, 50)) AS i_brand#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#11, 50)) AS i_product_name#15] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] (11) BroadcastExchange Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] @@ -97,28 +97,28 @@ ReadSchema: struct Input [1]: [w_warehouse_sk#16] Condition : isnotnull(w_warehouse_sk#16) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [w_warehouse_sk#16] (17) BroadcastExchange Input [1]: [w_warehouse_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#16] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] -(20) Expand [codegen id : 4] +(20) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] Arguments: [[inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#3, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] @@ -129,10 +129,10 @@ Results [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_gr Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] Arguments: hashpartitioning(i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 5] +(23) CometNativeColumnarToRow Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] Functions [1]: [avg(inv_quantity_on_hand#3)] @@ -147,7 +147,7 @@ Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.date_dim (26) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_mo Input [2]: [d_date_sk#6, d_month_seq#28] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt index 9b15a52cff..e1766e64d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -15,22 +15,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/simplified.txt index 092e187177..baff9284b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,26 +19,20 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometNativeColumnarToRow + CometFilter [w_warehouse_sk] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/explain.txt index c45ad5f2fa..448ca47190 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/explain.txt @@ -147,7 +147,7 @@ Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt index 7369619d2f..34e673af7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/simplified.txt index 9119ee749d..2cc520ff12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/simplified.txt @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt index c45ad5f2fa..448ca47190 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt @@ -147,7 +147,7 @@ Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/extended.txt index 7369619d2f..34e673af7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 9119ee749d..2cc520ff12 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 @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/explain.txt index 029c7fd3df..150428e457 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (69) +CometNativeColumnarToRow (69) +- CometHashAggregate (68) +- CometExchange (67) +- CometHashAggregate (66) @@ -18,7 +18,7 @@ : : : +- * Project (20) : : : +- * Filter (19) : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) + : : : +- CometNativeColumnarToRow (17) : : : +- CometColumnarExchange (16) : : : +- * HashAggregate (15) : : : +- * Project (14) @@ -30,7 +30,7 @@ : : : : : +- Scan parquet spark_catalog.default.store_sales (3) : : : : +- ReusedExchange (6) : : : +- BroadcastExchange (12) - : : : +- * CometColumnarToRow (11) + : : : +- CometNativeColumnarToRow (11) : : : +- CometFilter (10) : : : +- CometNativeScan parquet spark_catalog.default.item (9) : : +- CometSort (39) @@ -77,7 +77,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (3) Scan parquet spark_catalog.default.store_sales @@ -88,23 +88,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(4) ColumnarToRow [codegen id : 3] +(4) ColumnarToRow [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(5) Filter [codegen id : 3] +(5) Filter [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) (6) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#10, d_date#11] -(7) BroadcastHashJoin [codegen id : 3] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(8) Project [codegen id : 3] +(8) Project [codegen id : 1] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] @@ -119,24 +119,24 @@ ReadSchema: struct Input [2]: [i_item_sk#12, i_item_desc#13] Condition : isnotnull(i_item_sk#12) -(11) CometColumnarToRow [codegen id : 2] +(11) CometNativeColumnarToRow Input [2]: [i_item_sk#12, i_item_desc#13] (12) BroadcastExchange Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 1] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 1] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] @@ -147,21 +147,21 @@ Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(19) Filter [codegen id : 4] +(19) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(20) Project [codegen id : 4] +(20) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] @@ -169,13 +169,13 @@ Input [2]: [item_sk#18, cnt#19] Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 5] +(22) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(23) Project [codegen id : 5] +(23) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -296,19 +296,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#6)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 10] +(49) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] (50) ReusedExchange [Reuses operator id: 21] Output [1]: [item_sk#39] -(51) BroadcastHashJoin [codegen id : 10] +(51) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#34] Right keys [1]: [item_sk#39] Join type: LeftSemi Join condition: None -(52) Project [codegen id : 10] +(52) Project [codegen id : 6] Output [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] @@ -379,14 +379,14 @@ Input [2]: [sum#48, isEmpty#49] Keys: [] Functions [1]: [sum(sales#33)] -(69) CometColumnarToRow [codegen id : 11] +(69) CometNativeColumnarToRow Input [1]: [sum(sales)#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometProject (72) +- CometFilter (71) +- CometNativeScan parquet spark_catalog.default.date_dim (70) @@ -407,7 +407,7 @@ Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2 Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Arguments: [d_date_sk#30], [d_date_sk#30] -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [1]: [d_date_sk#30] (74) BroadcastExchange @@ -416,7 +416,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometFilter (76) +- CometNativeScan parquet spark_catalog.default.date_dim (75) @@ -437,7 +437,7 @@ Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#51] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (79) BroadcastExchange @@ -446,11 +446,11 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#28, [id=#29] * HashAggregate (99) -+- * CometColumnarToRow (98) ++- CometNativeColumnarToRow (98) +- CometColumnarExchange (97) +- * HashAggregate (96) +- * HashAggregate (95) - +- * CometColumnarToRow (94) + +- CometNativeColumnarToRow (94) +- CometColumnarExchange (93) +- * HashAggregate (92) +- * Project (91) @@ -461,7 +461,7 @@ Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquer : : +- * ColumnarToRow (81) : : +- Scan parquet spark_catalog.default.store_sales (80) : +- BroadcastExchange (86) - : +- * CometColumnarToRow (85) + : +- CometNativeColumnarToRow (85) : +- CometFilter (84) : +- CometNativeScan parquet spark_catalog.default.customer (83) +- ReusedExchange (89) @@ -475,10 +475,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 3] +(81) ColumnarToRow [codegen id : 1] Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -(82) Filter [codegen id : 3] +(82) Filter [codegen id : 1] Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] Condition : isnotnull(ss_customer_sk#52) @@ -493,37 +493,37 @@ ReadSchema: struct Input [1]: [c_customer_sk#57] Condition : isnotnull(c_customer_sk#57) -(85) CometColumnarToRow [codegen id : 1] +(85) CometNativeColumnarToRow Input [1]: [c_customer_sk#57] (86) BroadcastExchange Input [1]: [c_customer_sk#57] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(87) BroadcastHashJoin [codegen id : 3] +(87) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#52] Right keys [1]: [c_customer_sk#57] Join type: Inner Join condition: None -(88) Project [codegen id : 3] +(88) Project [codegen id : 1] Output [4]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] Input [5]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] (89) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#58] -(90) BroadcastHashJoin [codegen id : 3] +(90) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#55] Right keys [1]: [d_date_sk#58] Join type: Inner Join condition: None -(91) Project [codegen id : 3] +(91) Project [codegen id : 1] Output [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] Input [5]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57, d_date_sk#58] -(92) HashAggregate [codegen id : 3] +(92) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] Keys [1]: [c_customer_sk#57] Functions [1]: [partial_sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] @@ -534,17 +534,17 @@ Results [3]: [c_customer_sk#57, sum#61, isEmpty#62] Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] Arguments: hashpartitioning(c_customer_sk#57, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(94) CometColumnarToRow [codegen id : 4] +(94) CometNativeColumnarToRow Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -(95) HashAggregate [codegen id : 4] +(95) HashAggregate [codegen id : 2] Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] Keys [1]: [c_customer_sk#57] Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63] Results [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63 AS csales#64] -(96) HashAggregate [codegen id : 4] +(96) HashAggregate [codegen id : 2] Input [1]: [csales#64] Keys: [] Functions [1]: [partial_max(csales#64)] @@ -555,10 +555,10 @@ Results [1]: [max#66] Input [1]: [max#66] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(98) CometColumnarToRow [codegen id : 5] +(98) CometNativeColumnarToRow Input [1]: [max#66] -(99) HashAggregate [codegen id : 5] +(99) HashAggregate [codegen id : 3] Input [1]: [max#66] Keys: [] Functions [1]: [max(csales#64)] @@ -567,7 +567,7 @@ Results [1]: [max(csales#64)#67 AS tpcds_cmax#68] Subquery:4 Hosting operator id = 80 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#56 BroadcastExchange (104) -+- * CometColumnarToRow (103) ++- CometNativeColumnarToRow (103) +- CometProject (102) +- CometFilter (101) +- CometNativeScan parquet spark_catalog.default.date_dim (100) @@ -588,7 +588,7 @@ Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#58)) Input [2]: [d_date_sk#58, d_year#69] Arguments: [d_date_sk#58], [d_date_sk#58] -(103) CometColumnarToRow [codegen id : 1] +(103) CometNativeColumnarToRow Input [1]: [d_date_sk#58] (104) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt index aadeb13f7b..7040e78da1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -23,7 +23,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -35,17 +35,17 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort @@ -53,11 +53,11 @@ CometColumnarToRow : : +- CometFilter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -69,16 +69,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -112,7 +112,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -124,17 +124,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/simplified.txt index d42a6ba29e..1a34e6b5b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/simplified.txt @@ -1,142 +1,130 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - 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] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] +CometNativeColumnarToRow + CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] + CometExchange #1 + 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] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + WholeStageCodegen (3) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (1) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (3) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_customer_sk] #11 + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - 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] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #15 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [d_date_sk] #14 + BroadcastExchange #13 + CometNativeColumnarToRow + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + ReusedExchange [d_date_sk] #12 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometExchange [c_customer_sk] #8 + 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] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #9 + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_quantity,ws_list_price] [sales] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #15 + WholeStageCodegen (6) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/explain.txt index ce21e141e4..1daaeb8fae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/explain.txt @@ -377,7 +377,7 @@ Input [1]: [sum(sales)#50] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 Input [3]: [d_date_sk#29, d_year#30, d_moy#31] Arguments: [d_date_sk#29], [d_date_sk#29] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -541,7 +541,7 @@ Input [1]: [tpcds_cmax#63] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 BroadcastExchange (100) -+- * CometColumnarToRow (99) ++- CometNativeColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) Input [2]: [d_date_sk#57, d_year#58] Arguments: [d_date_sk#57], [d_date_sk#57] -(99) CometColumnarToRow [codegen id : 1] +(99) CometNativeColumnarToRow Input [1]: [d_date_sk#57] (100) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt index a84226b077..0d7391c0d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,7 +112,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/simplified.txt index 8f1bddf6c0..38f04e8e26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/simplified.txt @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -36,12 +34,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt index ce21e141e4..1daaeb8fae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt @@ -377,7 +377,7 @@ Input [1]: [sum(sales)#50] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 Input [3]: [d_date_sk#29, d_year#30, d_moy#31] Arguments: [d_date_sk#29], [d_date_sk#29] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -541,7 +541,7 @@ Input [1]: [tpcds_cmax#63] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 BroadcastExchange (100) -+- * CometColumnarToRow (99) ++- CometNativeColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) Input [2]: [d_date_sk#57, d_year#58] Arguments: [d_date_sk#57], [d_date_sk#57] -(99) CometColumnarToRow [codegen id : 1] +(99) CometNativeColumnarToRow Input [1]: [d_date_sk#57] (100) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/extended.txt index a84226b077..0d7391c0d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,7 +112,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 8f1bddf6c0..38f04e8e26 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 @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -36,12 +34,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/explain.txt index ff47906a58..1b806edbc8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (90) +CometNativeColumnarToRow (90) +- CometTakeOrderedAndProject (89) +- CometUnion (88) :- CometHashAggregate (64) @@ -21,7 +21,7 @@ : : : : +- * Project (21) : : : : +- * Filter (20) : : : : +- * HashAggregate (19) - : : : : +- * CometColumnarToRow (18) + : : : : +- CometNativeColumnarToRow (18) : : : : +- CometColumnarExchange (17) : : : : +- * HashAggregate (16) : : : : +- * Project (15) @@ -33,7 +33,7 @@ : : : : : : +- Scan parquet spark_catalog.default.store_sales (4) : : : : : +- ReusedExchange (7) : : : : +- BroadcastExchange (13) - : : : : +- * CometColumnarToRow (12) + : : : : +- CometNativeColumnarToRow (12) : : : : +- CometFilter (11) : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : +- CometSort (40) @@ -99,10 +99,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Condition : isnotnull(cs_bill_customer_sk#1) @@ -114,23 +114,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 3] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(6) Filter [codegen id : 3] +(6) Filter [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) (7) ReusedExchange [Reuses operator id: 100] Output [2]: [d_date_sk#10, d_date#11] -(8) BroadcastHashJoin [codegen id : 3] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(9) Project [codegen id : 3] +(9) Project [codegen id : 1] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] @@ -145,24 +145,24 @@ ReadSchema: struct Input [2]: [i_item_sk#12, i_item_desc#13] Condition : isnotnull(i_item_sk#12) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [i_item_sk#12, i_item_desc#13] (13) BroadcastExchange Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(14) BroadcastHashJoin [codegen id : 3] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 3] +(15) Project [codegen id : 1] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(16) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 1] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] @@ -173,21 +173,21 @@ Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(18) CometColumnarToRow [codegen id : 4] +(18) CometNativeColumnarToRow Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 2] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(20) Filter [codegen id : 4] +(20) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(21) Project [codegen id : 4] +(21) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] @@ -195,13 +195,13 @@ Input [2]: [item_sk#18, cnt#19] Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 5] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 5] +(24) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -394,23 +394,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 10] +(66) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -(67) Filter [codegen id : 10] +(67) Filter [codegen id : 6] Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] Condition : isnotnull(ws_bill_customer_sk#41) (68) ReusedExchange [Reuses operator id: 22] Output [1]: [item_sk#45] -(69) BroadcastHashJoin [codegen id : 10] +(69) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#40] Right keys [1]: [item_sk#45] Join type: LeftSemi Join condition: None -(70) Project [codegen id : 10] +(70) Project [codegen id : 6] Output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] @@ -493,14 +493,14 @@ Child 1 Input [3]: [c_last_name#54, c_first_name#53, sales#59] Input [3]: [c_last_name#34, c_first_name#33, sales#58] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#34 ASC NULLS FIRST,c_first_name#33 ASC NULLS FIRST,sales#58 ASC NULLS FIRST], output=[c_last_name#34,c_first_name#33,sales#58]), [c_last_name#34, c_first_name#33, sales#58], 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, sales#58] -(90) CometColumnarToRow [codegen id : 11] +(90) CometNativeColumnarToRow Input [3]: [c_last_name#34, c_first_name#33, sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (95) -+- * CometColumnarToRow (94) ++- CometNativeColumnarToRow (94) +- CometProject (93) +- CometFilter (92) +- CometNativeScan parquet spark_catalog.default.date_dim (91) @@ -521,7 +521,7 @@ Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2 Input [3]: [d_date_sk#35, d_year#36, d_moy#37] Arguments: [d_date_sk#35], [d_date_sk#35] -(94) CometColumnarToRow [codegen id : 1] +(94) CometNativeColumnarToRow Input [1]: [d_date_sk#35] (95) BroadcastExchange @@ -530,7 +530,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (100) -+- * CometColumnarToRow (99) ++- CometNativeColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometNativeScan parquet spark_catalog.default.date_dim (96) @@ -551,7 +551,7 @@ Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#60] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(99) CometColumnarToRow [codegen id : 1] +(99) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (100) BroadcastExchange @@ -560,11 +560,11 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#28, [id=#29] * HashAggregate (120) -+- * CometColumnarToRow (119) ++- CometNativeColumnarToRow (119) +- CometColumnarExchange (118) +- * HashAggregate (117) +- * HashAggregate (116) - +- * CometColumnarToRow (115) + +- CometNativeColumnarToRow (115) +- CometColumnarExchange (114) +- * HashAggregate (113) +- * Project (112) @@ -575,7 +575,7 @@ Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquer : : +- * ColumnarToRow (102) : : +- Scan parquet spark_catalog.default.store_sales (101) : +- BroadcastExchange (107) - : +- * CometColumnarToRow (106) + : +- CometNativeColumnarToRow (106) : +- CometFilter (105) : +- CometNativeScan parquet spark_catalog.default.customer (104) +- ReusedExchange (110) @@ -589,10 +589,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#64), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 3] +(102) ColumnarToRow [codegen id : 1] Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] -(103) Filter [codegen id : 3] +(103) Filter [codegen id : 1] Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] Condition : isnotnull(ss_customer_sk#61) @@ -607,37 +607,37 @@ ReadSchema: struct Input [1]: [c_customer_sk#66] Condition : isnotnull(c_customer_sk#66) -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [c_customer_sk#66] (107) BroadcastExchange Input [1]: [c_customer_sk#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(108) BroadcastHashJoin [codegen id : 3] +(108) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#61] Right keys [1]: [c_customer_sk#66] Join type: Inner Join condition: None -(109) Project [codegen id : 3] +(109) Project [codegen id : 1] Output [4]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] Input [5]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] (110) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#67] -(111) BroadcastHashJoin [codegen id : 3] +(111) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#64] Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None -(112) Project [codegen id : 3] +(112) Project [codegen id : 1] Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] Input [5]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66, d_date_sk#67] -(113) HashAggregate [codegen id : 3] +(113) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] Keys [1]: [c_customer_sk#66] Functions [1]: [partial_sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] @@ -648,17 +648,17 @@ Results [3]: [c_customer_sk#66, sum#70, isEmpty#71] Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] Arguments: hashpartitioning(c_customer_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(115) CometColumnarToRow [codegen id : 4] +(115) CometNativeColumnarToRow Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] -(116) HashAggregate [codegen id : 4] +(116) HashAggregate [codegen id : 2] Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] Keys [1]: [c_customer_sk#66] Functions [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] Aggregate Attributes [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72] Results [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72 AS csales#73] -(117) HashAggregate [codegen id : 4] +(117) HashAggregate [codegen id : 2] Input [1]: [csales#73] Keys: [] Functions [1]: [partial_max(csales#73)] @@ -669,10 +669,10 @@ Results [1]: [max#75] Input [1]: [max#75] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(119) CometColumnarToRow [codegen id : 5] +(119) CometNativeColumnarToRow Input [1]: [max#75] -(120) HashAggregate [codegen id : 5] +(120) HashAggregate [codegen id : 3] Input [1]: [max#75] Keys: [] Functions [1]: [max(csales#73)] @@ -681,7 +681,7 @@ Results [1]: [max(csales#73)#76 AS tpcds_cmax#77] Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65 BroadcastExchange (125) -+- * CometColumnarToRow (124) ++- CometNativeColumnarToRow (124) +- CometProject (123) +- CometFilter (122) +- CometNativeScan parquet spark_catalog.default.date_dim (121) @@ -702,7 +702,7 @@ Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#67)) Input [2]: [d_date_sk#67, d_year#78] Arguments: [d_date_sk#67], [d_date_sk#67] -(124) CometColumnarToRow [codegen id : 1] +(124) CometNativeColumnarToRow Input [1]: [d_date_sk#67] (125) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt index d41bf0802a..188775e7df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometUnion :- CometHashAggregate @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -26,7 +26,7 @@ CometColumnarToRow : : : : +- Project : : : : +- Filter : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project @@ -38,17 +38,17 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- CometSort @@ -56,11 +56,11 @@ CometColumnarToRow : : : +- CometFilter : : : : +- Subquery : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project @@ -72,16 +72,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -154,17 +154,17 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/simplified.txt index 41f01311f8..d543da014a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/simplified.txt @@ -1,164 +1,152 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - 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 [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] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + CometTakeOrderedAndProject [c_last_name,c_first_name,sales] + CometUnion [c_last_name,c_first_name,sales] + 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 [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] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + WholeStageCodegen (3) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (1) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (3) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_customer_sk] #11 + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - 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] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #14 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #15 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - CometBroadcastExchange [d_date_sk] #16 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] #17 - 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] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #18 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 + BroadcastExchange #13 + CometNativeColumnarToRow + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + ReusedExchange [d_date_sk] #12 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometExchange [c_customer_sk] #8 + 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] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #9 + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #14 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] + CometSort [c_customer_sk,c_first_name,c_last_name] + CometExchange [c_customer_sk] #15 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] CometSort [c_customer_sk] CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 - ReusedExchange [d_date_sk] #16 + CometBroadcastExchange [d_date_sk] #16 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + 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] #17 + 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] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #18 + WholeStageCodegen (6) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 + ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/explain.txt index c1131ab0a1..6ae343c29b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/explain.txt @@ -491,7 +491,7 @@ Input [3]: [c_last_name#33, c_first_name#32, sales#58] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (93) -+- * CometColumnarToRow (92) ++- CometNativeColumnarToRow (92) +- CometProject (91) +- CometFilter (90) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) @@ -512,7 +512,7 @@ Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2 Input [3]: [d_date_sk#34, d_year#35, d_moy#36] Arguments: [d_date_sk#34], [d_date_sk#34] -(92) CometColumnarToRow [codegen id : 1] +(92) CometNativeColumnarToRow Input [1]: [d_date_sk#34] (93) BroadcastExchange @@ -521,7 +521,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometProject (96) +- CometFilter (95) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) @@ -542,7 +542,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (98) BroadcastExchange @@ -655,7 +655,7 @@ Input [1]: [tpcds_cmax#72] Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -676,7 +676,7 @@ Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) Input [2]: [d_date_sk#66, d_year#67] Arguments: [d_date_sk#66], [d_date_sk#66] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#66] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt index cd91f5de49..4838389d3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,7 +35,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/simplified.txt index 92563114ae..b81ee0b822 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -39,12 +37,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -73,12 +69,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt index c1131ab0a1..6ae343c29b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt @@ -491,7 +491,7 @@ Input [3]: [c_last_name#33, c_first_name#32, sales#58] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (93) -+- * CometColumnarToRow (92) ++- CometNativeColumnarToRow (92) +- CometProject (91) +- CometFilter (90) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) @@ -512,7 +512,7 @@ Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2 Input [3]: [d_date_sk#34, d_year#35, d_moy#36] Arguments: [d_date_sk#34], [d_date_sk#34] -(92) CometColumnarToRow [codegen id : 1] +(92) CometNativeColumnarToRow Input [1]: [d_date_sk#34] (93) BroadcastExchange @@ -521,7 +521,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometProject (96) +- CometFilter (95) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) @@ -542,7 +542,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (98) BroadcastExchange @@ -655,7 +655,7 @@ Input [1]: [tpcds_cmax#72] Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -676,7 +676,7 @@ Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) Input [2]: [d_date_sk#66, d_year#67] Arguments: [d_date_sk#66], [d_date_sk#66] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#66] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/extended.txt index cd91f5de49..4838389d3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,7 +35,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 92563114ae..b81ee0b822 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -39,12 +37,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -73,12 +69,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/explain.txt index d6087012db..183e8089e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == * Filter (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) + :- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometProject (24) @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometNativeColumnarToRow Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometNativeScan parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCod Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometNativeColumnarToRow Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -272,16 +272,16 @@ Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subqu Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometColumnarExchange (75) +- * HashAggregate (74) +- * HashAggregate (73) - +- * CometColumnarToRow (72) + +- CometNativeColumnarToRow (72) +- CometColumnarExchange (71) +- * HashAggregate (70) +- * Project (69) +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) + :- CometNativeColumnarToRow (66) : +- CometProject (65) : +- CometBroadcastHashJoin (64) : :- CometProject (62) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] -(66) CometColumnarToRow [codegen id : 2] +(66) CometNativeColumnarToRow Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#73, ca_zip#74, ca_country#75] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#72, s_zip#59] Right keys [2]: [upper(ca_country#75), ca_zip#74] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_ Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometNativeColumnarToRow Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#78] Keys: [] Functions [1]: [partial_avg(netpaid#78)] @@ -426,10 +426,10 @@ Results [2]: [sum#81, count#82] Input [2]: [sum#81, count#82] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometNativeColumnarToRow Input [2]: [sum#81, count#82] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(netpaid#78)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/extended.txt index b65f56f327..03e6809902 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/extended.txt @@ -1,16 +1,16 @@ Filter : +- Subquery : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow +: :- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometProject @@ -42,21 +42,21 @@ Filter : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometProject @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/simplified.txt index af8d5ee7aa..affed709e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -39,21 +39,21 @@ WholeStageCodegen (4) InputAdapter ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/explain.txt index df3b1ff6b0..1088bc04e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCod Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#73, ca_zip#74, ca_country#75] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#72, s_zip#59] Right keys [2]: [upper(ca_country#75), ca_zip#74] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_ Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#78] Keys: [] Functions [1]: [partial_avg(netpaid#78)] @@ -426,10 +426,10 @@ Results [2]: [sum#81, count#82] Input [2]: [sum#81, count#82] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#81, count#82] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(netpaid#78)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt index df3b1ff6b0..1088bc04e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCod Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#73, ca_zip#74, ca_country#75] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#72, s_zip#59] Right keys [2]: [upper(ca_country#75), ca_zip#74] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_ Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#78] Keys: [] Functions [1]: [partial_avg(netpaid#78)] @@ -426,10 +426,10 @@ Results [2]: [sum#81, count#82] Input [2]: [sum#81, count#82] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#81, count#82] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(netpaid#78)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/explain.txt index f4e4f60060..540d38da6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == * Filter (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) + :- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometProject (24) @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometNativeColumnarToRow Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometNativeScan parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCod Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometNativeColumnarToRow Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -272,16 +272,16 @@ Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subqu Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometColumnarExchange (75) +- * HashAggregate (74) +- * HashAggregate (73) - +- * CometColumnarToRow (72) + +- CometNativeColumnarToRow (72) +- CometColumnarExchange (71) +- * HashAggregate (70) +- * Project (69) +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) + :- CometNativeColumnarToRow (66) : +- CometProject (65) : +- CometBroadcastHashJoin (64) : :- CometProject (62) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] -(66) CometColumnarToRow [codegen id : 2] +(66) CometNativeColumnarToRow Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#73, ca_zip#74, ca_country#75] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#72, s_zip#59] Right keys [2]: [upper(ca_country#75), ca_zip#74] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_ Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometNativeColumnarToRow Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#78] Keys: [] Functions [1]: [partial_avg(netpaid#78)] @@ -426,10 +426,10 @@ Results [2]: [sum#81, count#82] Input [2]: [sum#81, count#82] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometNativeColumnarToRow Input [2]: [sum#81, count#82] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(netpaid#78)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/extended.txt index b65f56f327..03e6809902 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/extended.txt @@ -1,16 +1,16 @@ Filter : +- Subquery : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow +: :- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometProject @@ -42,21 +42,21 @@ Filter : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometProject @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/simplified.txt index af8d5ee7aa..affed709e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -39,21 +39,21 @@ WholeStageCodegen (4) InputAdapter ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/explain.txt index 4e088146fe..87b38cfb83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCod Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#73, ca_zip#74, ca_country#75] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#72, s_zip#59] Right keys [2]: [upper(ca_country#75), ca_zip#74] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_ Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#78] Keys: [] Functions [1]: [partial_avg(netpaid#78)] @@ -426,10 +426,10 @@ Results [2]: [sum#81, count#82] Input [2]: [sum#81, count#82] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#81, count#82] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(netpaid#78)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt index 4e088146fe..87b38cfb83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCod Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#73, ca_zip#74, ca_country#75] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#72, s_zip#59] Right keys [2]: [upper(ca_country#75), ca_zip#74] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_ Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#78] Keys: [] Functions [1]: [partial_avg(netpaid#78)] @@ -426,10 +426,10 @@ Results [2]: [sum#81, count#82] Input [2]: [sum#81, count#82] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#81, count#82] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#81, count#82] Keys: [] Functions [1]: [avg(netpaid#78)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/explain.txt index 36bfea5046..7f5072787a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- ReusedExchange (19) : : +- ReusedExchange (22) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.store (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.item (32) @@ -52,10 +52,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) @@ -78,13 +78,13 @@ Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnot Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 8] +(8) BroadcastHashJoin [codegen id : 3] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] Join type: Inner Join condition: None -(9) Project [codegen id : 8] +(9) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] @@ -107,52 +107,52 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 3] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] Join type: Inner Join condition: None -(15) Project [codegen id : 8] +(15) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] (16) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#18] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(18) Project [codegen id : 3] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#18] (19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#19] -(20) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_returned_date_sk#12] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(21) Project [codegen id : 8] +(21) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] (22) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(24) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] @@ -171,20 +171,20 @@ Condition : isnotnull(s_store_sk#21) Input [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] Arguments: [s_store_sk#21, s_store_id#24, s_store_name#23], [s_store_sk#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#22, 16)) AS s_store_id#24, s_store_name#23] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] (29) BroadcastExchange Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#21, s_store_id#24, s_store_name#23] @@ -203,24 +203,24 @@ Condition : isnotnull(i_item_sk#25) Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] Arguments: [i_item_sk#25, i_item_id#28, i_item_desc#27], [i_item_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#26, 16)) AS i_item_id#28, i_item_desc#27] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] (36) BroadcastExchange Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#25] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(38) Project [codegen id : 3] Output [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_sk#25, i_item_id#28, i_item_desc#27] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 3] Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] @@ -231,10 +231,10 @@ Results [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum# Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] Arguments: hashpartitioning(i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 4] Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] @@ -249,7 +249,7 @@ Arguments: 100, [i_item_id#28 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 4) Input [3]: [d_date_sk#18, d_year#41, d_moy#42] Arguments: [d_date_sk#18], [d_date_sk#18] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#18] (48) BroadcastExchange @@ -279,7 +279,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -300,7 +300,7 @@ Condition : (((((isnotnull(d_moy#44) AND isnotnull(d_year#43)) AND (d_moy#44 >= Input [3]: [d_date_sk#19, d_year#43, d_moy#44] Arguments: [d_date_sk#19], [d_date_sk#19] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt index 26e924fd7e..15c6bec5dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,27 +42,27 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/simplified.txt index 288561a740..cc47e03799 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,12 +39,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -63,17 +59,13 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk,i_item_id,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt index f076c7886b..c43a59a570 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt @@ -256,7 +256,7 @@ Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt index dcad304452..7d8d4694e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt index f076c7886b..c43a59a570 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt @@ -256,7 +256,7 @@ Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 dcad304452..7d8d4694e8 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 @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/explain.txt index 5e1cd8b012..ee3dd93604 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * HashAggregate (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * Project (27) @@ -16,18 +16,18 @@ TakeOrderedAndProject (32) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : +- ReusedExchange (11) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.item (14) +- BroadcastExchange (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometNativeScan parquet spark_catalog.default.promotion (21) @@ -41,10 +41,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) @@ -63,33 +63,33 @@ Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) A Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#1] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] @@ -108,20 +108,20 @@ Condition : isnotnull(i_item_sk#15) Input [2]: [i_item_sk#15, i_item_id#16] Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#16, 16)) AS i_item_id#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [i_item_sk#15, i_item_id#17] (18) BroadcastExchange Input [2]: [i_item_sk#15, i_item_id#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#2] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#15, i_item_id#17] @@ -140,24 +140,24 @@ Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_e Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Arguments: [p_promo_sk#18], [p_promo_sk#18] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [1]: [p_promo_sk#18] (25) BroadcastExchange Input [1]: [p_promo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_promo_sk#3] Right keys [1]: [p_promo_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] @@ -168,10 +168,10 @@ Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34 Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Keys [1]: [i_item_id#17] Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] @@ -186,7 +186,7 @@ Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#45] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt index 3b9d40795e..ceba8f5828 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -16,27 +16,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/simplified.txt index 7eb9c67611..23b7e6a8f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] BroadcastHashJoin [cs_promo_sk,p_promo_sk] @@ -20,35 +20,27 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometNativeColumnarToRow + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/explain.txt index 06138ff307..1082e62019 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt index dbea7afe34..ea785136aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/simplified.txt index 0b56a47547..f040250da3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/simplified.txt @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt index 06138ff307..1082e62019 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/extended.txt index dbea7afe34..ea785136aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0b56a47547..f040250da3 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 @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/explain.txt index f3815e677f..5b96e870f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (33) +- * HashAggregate (32) - +- * CometColumnarToRow (31) + +- CometNativeColumnarToRow (31) +- CometColumnarExchange (30) +- * HashAggregate (29) +- * Expand (28) @@ -17,18 +17,18 @@ TakeOrderedAndProject (33) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : +- ReusedExchange (11) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.store (14) +- BroadcastExchange (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometNativeScan parquet spark_catalog.default.item (21) @@ -42,10 +42,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) @@ -64,33 +64,33 @@ Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) A Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 38] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -109,20 +109,20 @@ Condition : ((isnotnull(s_state#16) AND (static_invoke(CharVarcharCodegenUtils.r Input [2]: [s_store_sk#15, s_state#16] Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#16, 2)) AS s_state#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [s_store_sk#15, s_state#17] (18) BroadcastExchange Input [2]: [s_store_sk#15, s_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] @@ -141,28 +141,28 @@ Condition : isnotnull(i_item_sk#18) Input [2]: [i_item_sk#18, i_item_id#19] Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#19, 16)) AS i_item_id#20] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [2]: [i_item_sk#18, i_item_id#20] (25) BroadcastExchange Input [2]: [i_item_sk#18, i_item_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] -(28) Expand [codegen id : 5] +(28) Expand [codegen id : 1] Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] -(29) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 1] Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] @@ -173,10 +173,10 @@ Results [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] Arguments: hashpartitioning(i_item_id#21, s_state#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(31) CometColumnarToRow [codegen id : 6] +(31) CometNativeColumnarToRow Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 2] Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] @@ -191,7 +191,7 @@ Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#49] Arguments: [d_date_sk#14], [d_date_sk#14] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt index a4946e4771..20f7517d3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -17,27 +17,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/simplified.txt index dcb4953707..452cc03fb3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] @@ -21,35 +21,27 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/explain.txt index 8dc9e12f34..70787411b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/explain.txt @@ -186,7 +186,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt index f550f3855c..bbfc91ed08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/simplified.txt index bdae0cc477..98686d7452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt index 8dc9e12f34..70787411b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt @@ -186,7 +186,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/extended.txt index f550f3855c..bbfc91ed08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 bdae0cc477..98686d7452 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/explain.txt index 985274408b..bb87d4cc2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/explain.txt @@ -4,72 +4,72 @@ : :- * BroadcastNestedLoopJoin Inner BuildRight (50) : : :- * BroadcastNestedLoopJoin Inner BuildRight (37) : : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) +: : : : :- CometNativeColumnarToRow (11) : : : : : +- CometHashAggregate (10) : : : : : +- CometColumnarExchange (9) : : : : : +- * HashAggregate (8) : : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) +: : : : : +- CometNativeColumnarToRow (6) : : : : : +- CometExchange (5) : : : : : +- CometHashAggregate (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) : : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) +: : : : +- CometNativeColumnarToRow (22) : : : : +- CometHashAggregate (21) : : : : +- CometColumnarExchange (20) : : : : +- * HashAggregate (19) : : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) +: : : : +- CometNativeColumnarToRow (17) : : : : +- CometExchange (16) : : : : +- CometHashAggregate (15) : : : : +- CometProject (14) : : : : +- CometFilter (13) : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (12) : : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) +: : : +- CometNativeColumnarToRow (35) : : : +- CometHashAggregate (34) : : : +- CometColumnarExchange (33) : : : +- * HashAggregate (32) : : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) +: : : +- CometNativeColumnarToRow (30) : : : +- CometExchange (29) : : : +- CometHashAggregate (28) : : : +- CometProject (27) : : : +- CometFilter (26) : : : +- CometNativeScan parquet spark_catalog.default.store_sales (25) : : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) +: : +- CometNativeColumnarToRow (48) : : +- CometHashAggregate (47) : : +- CometColumnarExchange (46) : : +- * HashAggregate (45) : : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) +: : +- CometNativeColumnarToRow (43) : : +- CometExchange (42) : : +- CometHashAggregate (41) : : +- CometProject (40) : : +- CometFilter (39) : : +- CometNativeScan parquet spark_catalog.default.store_sales (38) : +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) +: +- CometNativeColumnarToRow (61) : +- CometHashAggregate (60) : +- CometColumnarExchange (59) : +- * HashAggregate (58) : +- * HashAggregate (57) -: +- * CometColumnarToRow (56) +: +- CometNativeColumnarToRow (56) : +- CometExchange (55) : +- CometHashAggregate (54) : +- CometProject (53) : +- CometFilter (52) : +- CometNativeScan parquet spark_catalog.default.store_sales (51) +- BroadcastExchange (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometHashAggregate (73) +- CometColumnarExchange (72) +- * HashAggregate (71) +- * HashAggregate (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometExchange (68) +- CometHashAggregate (67) +- CometProject (66) @@ -101,7 +101,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_li Input [4]: [ss_list_price#3, sum#6, count#7, count#8] Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [4]: [ss_list_price#3, sum#6, count#7, count#8] (7) HashAggregate [codegen id : 1] @@ -127,7 +127,7 @@ Input [4]: [sum#6, count#7, count#8, count#12] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -(11) CometColumnarToRow [codegen id : 12] +(11) CometNativeColumnarToRow Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] (12) CometNativeScan parquet spark_catalog.default.store_sales @@ -154,7 +154,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_l Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(17) CometColumnarToRow [codegen id : 2] +(17) CometNativeColumnarToRow Input [4]: [ss_list_price#18, sum#21, count#22, count#23] (18) HashAggregate [codegen id : 2] @@ -180,14 +180,14 @@ Input [4]: [sum#21, count#22, count#23, count#27] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] -(22) CometColumnarToRow [codegen id : 3] +(22) CometNativeColumnarToRow Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] (23) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(24) BroadcastNestedLoopJoin [codegen id : 12] +(24) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -215,17 +215,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_l Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(30) CometColumnarToRow [codegen id : 4] +(30) CometNativeColumnarToRow Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -(31) HashAggregate [codegen id : 4] +(31) HashAggregate [codegen id : 3] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys [1]: [ss_list_price#33] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] Results [4]: [ss_list_price#33, sum#36, count#37, count#38] -(32) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 3] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] @@ -241,14 +241,14 @@ Input [4]: [sum#36, count#37, count#38, count#42] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] -(35) CometColumnarToRow [codegen id : 5] +(35) CometNativeColumnarToRow Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] (36) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(37) BroadcastNestedLoopJoin [codegen id : 12] +(37) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -276,17 +276,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_l Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(43) CometColumnarToRow [codegen id : 6] +(43) CometNativeColumnarToRow Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -(44) HashAggregate [codegen id : 6] +(44) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys [1]: [ss_list_price#48] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] Results [4]: [ss_list_price#48, sum#51, count#52, count#53] -(45) HashAggregate [codegen id : 6] +(45) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] @@ -302,14 +302,14 @@ Input [4]: [sum#51, count#52, count#53, count#57] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] -(48) CometColumnarToRow [codegen id : 7] +(48) CometNativeColumnarToRow Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] (49) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(50) BroadcastNestedLoopJoin [codegen id : 12] +(50) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -337,17 +337,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_l Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(56) CometColumnarToRow [codegen id : 8] +(56) CometNativeColumnarToRow Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -(57) HashAggregate [codegen id : 8] +(57) HashAggregate [codegen id : 5] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys [1]: [ss_list_price#63] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] Results [4]: [ss_list_price#63, sum#66, count#67, count#68] -(58) HashAggregate [codegen id : 8] +(58) HashAggregate [codegen id : 5] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] @@ -363,14 +363,14 @@ Input [4]: [sum#66, count#67, count#68, count#72] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] -(61) CometColumnarToRow [codegen id : 9] +(61) CometNativeColumnarToRow Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] (62) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(63) BroadcastNestedLoopJoin [codegen id : 12] +(63) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -398,17 +398,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_l Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(69) CometColumnarToRow [codegen id : 10] +(69) CometNativeColumnarToRow Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -(70) HashAggregate [codegen id : 10] +(70) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys [1]: [ss_list_price#78] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] Results [4]: [ss_list_price#78, sum#81, count#82, count#83] -(71) HashAggregate [codegen id : 10] +(71) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] @@ -424,14 +424,14 @@ Input [4]: [sum#81, count#82, count#83, count#87] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] -(74) CometColumnarToRow [codegen id : 11] +(74) CometNativeColumnarToRow Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] (75) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(76) BroadcastNestedLoopJoin [codegen id : 12] +(76) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/extended.txt index 37c297f68a..d05989eb4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/extended.txt @@ -3,72 +3,72 @@ BroadcastNestedLoopJoin : :- BroadcastNestedLoopJoin : : :- BroadcastNestedLoopJoin : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : :- CometColumnarToRow +: : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate : : : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : : +- CometColumnarToRow +: : : : : +- CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometHashAggregate : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- BroadcastExchange -: : : : +- CometColumnarToRow +: : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : +- CometColumnarToRow +: : : : +- CometNativeColumnarToRow : : : : +- CometExchange : : : : +- CometHashAggregate : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- BroadcastExchange -: : : +- CometColumnarToRow +: : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : +- CometColumnarToRow +: : : +- CometNativeColumnarToRow : : : +- CometExchange : : : +- CometHashAggregate : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- BroadcastExchange -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometExchange : : +- CometHashAggregate : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometExchange : +- CometHashAggregate : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometExchange +- CometHashAggregate +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/simplified.txt index a8540a4abb..96a591d9ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (12) +WholeStageCodegen (7) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] CometColumnarExchange #1 WholeStageCodegen (1) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometExchange [ss_list_price] #2 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] @@ -20,86 +20,76 @@ WholeStageCodegen (12) CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #4 + WholeStageCodegen (2) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #5 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #7 + WholeStageCodegen (3) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #8 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) + InputAdapter + BroadcastExchange #9 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #10 + WholeStageCodegen (4) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #11 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #12 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #13 + WholeStageCodegen (5) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #14 + CometHashAggregate [ss_list_price,sum,count,count] + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #15 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #16 + WholeStageCodegen (6) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #17 + CometHashAggregate [ss_list_price,sum,count,count] + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/explain.txt index f8b79c2e07..5981690e83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- ReusedExchange (19) : : +- ReusedExchange (22) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.store (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.item (32) @@ -52,10 +52,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) @@ -78,13 +78,13 @@ Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnot Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 8] +(8) BroadcastHashJoin [codegen id : 3] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] Join type: Inner Join condition: None -(9) Project [codegen id : 8] +(9) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] @@ -107,52 +107,52 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 3] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] Join type: Inner Join condition: None -(15) Project [codegen id : 8] +(15) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] (16) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#19] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(18) Project [codegen id : 3] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] (19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#20] -(20) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_returned_date_sk#12] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(21) Project [codegen id : 8] +(21) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] (22) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#21] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(24) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] @@ -171,20 +171,20 @@ Condition : isnotnull(s_store_sk#22) Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] Arguments: [s_store_sk#22, s_store_id#25, s_store_name#24], [s_store_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#23, 16)) AS s_store_id#25, s_store_name#24] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] (29) BroadcastExchange Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#22] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_store_id#25, s_store_name#24] @@ -203,24 +203,24 @@ Condition : isnotnull(i_item_sk#26) Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] Arguments: [i_item_sk#26, i_item_id#29, i_item_desc#28], [i_item_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#27, 16)) AS i_item_id#29, i_item_desc#28] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] (36) BroadcastExchange Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(38) Project [codegen id : 3] Output [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_sk#26, i_item_id#29, i_item_desc#28] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 3] Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] @@ -231,10 +231,10 @@ Results [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum# Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] Arguments: hashpartitioning(i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 4] Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] @@ -249,7 +249,7 @@ Arguments: 100, [i_item_id#29 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 9) Input [3]: [d_date_sk#19, d_year#42, d_moy#43] Arguments: [d_date_sk#19], [d_date_sk#19] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (48) BroadcastExchange @@ -279,7 +279,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -300,7 +300,7 @@ Condition : (((((isnotnull(d_moy#45) AND isnotnull(d_year#44)) AND (d_moy#45 >= Input [3]: [d_date_sk#20, d_year#44, d_moy#45] Arguments: [d_date_sk#20], [d_date_sk#20] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#20] (53) BroadcastExchange @@ -309,7 +309,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (58) -+- * CometColumnarToRow (57) ++- CometNativeColumnarToRow (57) +- CometProject (56) +- CometFilter (55) +- CometNativeScan parquet spark_catalog.default.date_dim (54) @@ -330,7 +330,7 @@ Condition : (d_year#46 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) Input [2]: [d_date_sk#21, d_year#46] Arguments: [d_date_sk#21], [d_date_sk#21] -(57) CometColumnarToRow [codegen id : 1] +(57) CometNativeColumnarToRow Input [1]: [d_date_sk#21] (58) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt index dacaab515f..bd9fa52aa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,32 +42,32 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/simplified.txt index 54bdbeeb5a..19ffcce191 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,12 +39,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -56,12 +52,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -70,17 +64,13 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk,i_item_id,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/explain.txt index a885286877..0a745e69f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/explain.txt @@ -275,7 +275,7 @@ Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -296,7 +296,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (52) BroadcastExchange @@ -305,7 +305,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -326,7 +326,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (57) BroadcastExchange @@ -335,7 +335,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometFilter (59) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) @@ -356,7 +356,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#25] (62) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt index fe03fef8a7..8c47903ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/simplified.txt index 15b992d3a4..e772ae6f33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/simplified.txt @@ -23,34 +23,28 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt index a885286877..0a745e69f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt @@ -275,7 +275,7 @@ Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -296,7 +296,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (52) BroadcastExchange @@ -305,7 +305,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -326,7 +326,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (57) BroadcastExchange @@ -335,7 +335,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometFilter (59) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) @@ -356,7 +356,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#25] (62) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/extended.txt index fe03fef8a7..8c47903ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 15b992d3a4..e772ae6f33 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 @@ -23,34 +23,28 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/explain.txt index 0fe8bceddd..05b309221e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/simplified.txt index c19e6caf52..0a6cce1eb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - 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 [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] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] + 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 [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] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/explain.txt index 87bbeb050a..d65e29a3d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (54) : : +- * BroadcastHashJoin Inner BuildRight (38) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) @@ -20,18 +20,18 @@ TakeOrderedAndProject (54) : : : : : +- Scan parquet spark_catalog.default.web_returns (1) : : : : +- ReusedExchange (4) : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) + : : : +- CometNativeColumnarToRow (10) : : : +- CometProject (9) : : : +- CometFilter (8) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : : +- BroadcastExchange (37) : : +- * Filter (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) + : : +- CometNativeColumnarToRow (30) : : +- CometColumnarExchange (29) : : +- * HashAggregate (28) : : +- * Project (27) @@ -44,12 +44,12 @@ TakeOrderedAndProject (54) : : : +- ReusedExchange (22) : : +- ReusedExchange (25) : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometProject (42) : +- CometFilter (41) : +- CometNativeScan parquet spark_catalog.default.customer (40) +- BroadcastExchange (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometNativeScan parquet spark_catalog.default.customer_address (47) @@ -63,23 +63,23 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [wr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] @@ -98,24 +98,24 @@ Condition : (isnotnull(ca_address_sk#7) AND isnotnull(static_invoke(CharVarcharC Input [2]: [ca_address_sk#7, ca_state#8] Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)) AS ca_state#9] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [ca_address_sk#7, ca_state#9] (11) BroadcastExchange Input [2]: [ca_address_sk#7, ca_state#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [wr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#9] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] @@ -126,17 +126,17 @@ Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 11] +(16) CometNativeColumnarToRow Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -(17) HashAggregate [codegen id : 11] +(17) HashAggregate [codegen id : 5] Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] -(18) Filter [codegen id : 11] +(18) Filter [codegen id : 5] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) @@ -148,40 +148,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#19), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 2] Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -(21) Filter [codegen id : 6] +(21) Filter [codegen id : 2] Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] Condition : isnotnull(wr_returning_addr_sk#17) (22) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(24) Project [codegen id : 2] Output [3]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19, d_date_sk#20] (25) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#21, ca_state#22] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_returning_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 2] Output [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#22] Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, ca_address_sk#21, ca_state#22] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#22] Keys [2]: [wr_returning_customer_sk#16, ca_state#22] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#18))] @@ -192,17 +192,17 @@ Results [3]: [wr_returning_customer_sk#16, ca_state#22, sum#24] Input [3]: [wr_returning_customer_sk#16, ca_state#22, sum#24] Arguments: hashpartitioning(wr_returning_customer_sk#16, ca_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [3]: [wr_returning_customer_sk#16, ca_state#22, sum#24] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [3]: [wr_returning_customer_sk#16, ca_state#22, sum#24] Keys [2]: [wr_returning_customer_sk#16, ca_state#22] Functions [1]: [sum(UnscaledValue(wr_return_amt#18))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#18))#12] Results [2]: [ca_state#22 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(wr_return_amt#18))#12,17,2) AS ctr_total_return#26] -(32) HashAggregate [codegen id : 7] +(32) HashAggregate [codegen id : 3] Input [2]: [ctr_state#25, ctr_total_return#26] Keys [1]: [ctr_state#25] Functions [1]: [partial_avg(ctr_total_return#26)] @@ -213,17 +213,17 @@ Results [3]: [ctr_state#25, sum#29, count#30] Input [3]: [ctr_state#25, sum#29, count#30] Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 8] +(34) CometNativeColumnarToRow Input [3]: [ctr_state#25, sum#29, count#30] -(35) HashAggregate [codegen id : 8] +(35) HashAggregate [codegen id : 4] Input [3]: [ctr_state#25, sum#29, count#30] Keys [1]: [ctr_state#25] Functions [1]: [avg(ctr_total_return#26)] Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] -(36) Filter [codegen id : 8] +(36) Filter [codegen id : 4] Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) @@ -231,13 +231,13 @@ Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 11] +(38) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_state#14] Right keys [1]: [ctr_state#25] Join type: Inner Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) -(39) Project [codegen id : 11] +(39) Project [codegen id : 5] Output [2]: [ctr_customer_sk#13, ctr_total_return#15] Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] @@ -256,20 +256,20 @@ Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] Arguments: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46], [c_customer_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#34, 16)) AS c_customer_id#47, c_current_addr_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#36, 10)) AS c_salutation#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#39, 1)) AS c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#44, 13)) AS c_login#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#45, 50)) AS c_email_address#53, c_last_review_date#46] -(43) CometColumnarToRow [codegen id : 9] +(43) CometNativeColumnarToRow Input [14]: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] (44) BroadcastExchange Input [14]: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_customer_sk#13] Right keys [1]: [c_customer_sk#33] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(46) Project [codegen id : 5] Output [14]: [ctr_total_return#15, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] Input [16]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] @@ -288,20 +288,20 @@ Condition : ((isnotnull(ca_state#55) AND (static_invoke(CharVarcharCodegenUtils. Input [2]: [ca_address_sk#54, ca_state#55] Arguments: [ca_address_sk#54], [ca_address_sk#54] -(50) CometColumnarToRow [codegen id : 10] +(50) CometNativeColumnarToRow Input [1]: [ca_address_sk#54] (51) BroadcastExchange Input [1]: [ca_address_sk#54] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#35] Right keys [1]: [ca_address_sk#54] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(53) Project [codegen id : 5] Output [13]: [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] Input [15]: [ctr_total_return#15, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ca_address_sk#54] @@ -313,7 +313,7 @@ Arguments: 100, [c_customer_id#47 ASC NULLS FIRST, c_salutation#48 ASC NULLS FIR Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (59) -+- * CometColumnarToRow (58) ++- CometNativeColumnarToRow (58) +- CometProject (57) +- CometFilter (56) +- CometNativeScan parquet spark_catalog.default.date_dim (55) @@ -334,7 +334,7 @@ Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#6, d_year#56] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(58) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (59) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt index 8373409822..c2e96eb412 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -19,28 +19,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -52,22 +52,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/simplified.txt index 72b6fbd1ba..4eacdafde1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (5) Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] @@ -8,10 +8,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] Project [wr_returning_customer_sk,wr_return_amt,ca_state] BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] @@ -23,37 +23,33 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre Scan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (8) + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] Project [wr_returning_customer_sk,wr_return_amt,ca_state] BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] @@ -70,17 +66,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometNativeColumnarToRow + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt index dac625d856..09880b4d2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt @@ -285,7 +285,7 @@ Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt index 7ebd69d340..70bff8abb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt index 0227534fa5..6eebcf2802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt index dac625d856..09880b4d2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt @@ -285,7 +285,7 @@ Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt index 7ebd69d340..70bff8abb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0227534fa5..6eebcf2802 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/explain.txt index 9ce8c738e8..24cf234506 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (97) +CometNativeColumnarToRow (97) +- CometSort (96) +- CometColumnarExchange (95) +- * Project (94) @@ -11,7 +11,7 @@ : : : +- * BroadcastHashJoin Inner BuildRight (46) : : : :- * BroadcastHashJoin Inner BuildRight (31) : : : : :- * HashAggregate (16) - : : : : : +- * CometColumnarToRow (15) + : : : : : +- CometNativeColumnarToRow (15) : : : : : +- CometColumnarExchange (14) : : : : : +- * HashAggregate (13) : : : : : +- * Project (12) @@ -23,12 +23,12 @@ : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : : +- ReusedExchange (4) : : : : : +- BroadcastExchange (10) - : : : : : +- * CometColumnarToRow (9) + : : : : : +- CometNativeColumnarToRow (9) : : : : : +- CometFilter (8) : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : : : : +- BroadcastExchange (30) : : : : +- * HashAggregate (29) - : : : : +- * CometColumnarToRow (28) + : : : : +- CometNativeColumnarToRow (28) : : : : +- CometColumnarExchange (27) : : : : +- * HashAggregate (26) : : : : +- * Project (25) @@ -42,7 +42,7 @@ : : : : +- ReusedExchange (23) : : : +- BroadcastExchange (45) : : : +- * HashAggregate (44) - : : : +- * CometColumnarToRow (43) + : : : +- CometNativeColumnarToRow (43) : : : +- CometColumnarExchange (42) : : : +- * HashAggregate (41) : : : +- * Project (40) @@ -56,7 +56,7 @@ : : : +- ReusedExchange (38) : : +- BroadcastExchange (61) : : +- * HashAggregate (60) - : : +- * CometColumnarToRow (59) + : : +- CometNativeColumnarToRow (59) : : +- CometColumnarExchange (58) : : +- * HashAggregate (57) : : +- * Project (56) @@ -70,7 +70,7 @@ : : +- ReusedExchange (54) : +- BroadcastExchange (76) : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) + : +- CometNativeColumnarToRow (74) : +- CometColumnarExchange (73) : +- * HashAggregate (72) : +- * Project (71) @@ -84,7 +84,7 @@ : +- ReusedExchange (69) +- BroadcastExchange (92) +- * HashAggregate (91) - +- * CometColumnarToRow (90) + +- CometNativeColumnarToRow (90) +- CometColumnarExchange (89) +- * HashAggregate (88) +- * Project (87) @@ -106,23 +106,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_addr_sk#1) (4) ReusedExchange [Reuses operator id: 101] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] @@ -137,24 +137,24 @@ ReadSchema: struct Input [2]: [ca_address_sk#8, ca_county#9] Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) -(9) CometColumnarToRow [codegen id : 2] +(9) CometNativeColumnarToRow Input [2]: [ca_address_sk#8, ca_county#9] (10) BroadcastExchange Input [2]: [ca_address_sk#8, ca_county#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#1] Right keys [1]: [ca_address_sk#8] Join type: Inner Join condition: None -(12) Project [codegen id : 3] +(12) Project [codegen id : 1] Output [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 1] Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -165,10 +165,10 @@ Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(15) CometColumnarToRow [codegen id : 24] +(15) CometNativeColumnarToRow Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -(16) HashAggregate [codegen id : 24] +(16) HashAggregate [codegen id : 12] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -183,40 +183,40 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Condition : isnotnull(ss_addr_sk#14) (20) ReusedExchange [Reuses operator id: 105] Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#16] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] (23) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#21, ca_county#22] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_addr_sk#14] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] @@ -227,10 +227,10 @@ Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -(29) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 3] Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] @@ -241,7 +241,7 @@ Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15) Input [2]: [ca_county#22, store_sales#25] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 24] +(31) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#22] Join type: Inner @@ -255,40 +255,40 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] +(33) ColumnarToRow [codegen id : 4] Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -(34) Filter [codegen id : 10] +(34) Filter [codegen id : 4] Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Condition : isnotnull(ss_addr_sk#26) (35) ReusedExchange [Reuses operator id: 109] Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#28] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 4] Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] (38) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#33, ca_county#34] -(39) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_addr_sk#26] Right keys [1]: [ca_address_sk#33] Join type: Inner Join condition: None -(40) Project [codegen id : 10] +(40) Project [codegen id : 4] Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] -(41) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 4] Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] @@ -299,10 +299,10 @@ Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometColumnarToRow [codegen id : 11] +(43) CometNativeColumnarToRow Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 5] Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] @@ -313,13 +313,13 @@ Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27) Input [2]: [ca_county#34, store_sales#37] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 24] +(46) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#22] Right keys [1]: [ca_county#34] Join type: Inner Join condition: None -(47) Project [codegen id : 24] +(47) Project [codegen id : 12] Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] @@ -331,40 +331,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 14] +(49) ColumnarToRow [codegen id : 6] Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(50) Filter [codegen id : 14] +(50) Filter [codegen id : 6] Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Condition : isnotnull(ws_bill_addr_sk#38) (51) ReusedExchange [Reuses operator id: 101] Output [3]: [d_date_sk#41, d_year#42, d_qoy#43] -(52) BroadcastHashJoin [codegen id : 14] +(52) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#40] Right keys [1]: [d_date_sk#41] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 6] Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43] Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41, d_year#42, d_qoy#43] (54) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#44, ca_county#45] -(55) BroadcastHashJoin [codegen id : 14] +(55) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_bill_addr_sk#38] Right keys [1]: [ca_address_sk#44] Join type: Inner Join condition: None -(56) Project [codegen id : 14] +(56) Project [codegen id : 6] Output [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_address_sk#44, ca_county#45] -(57) HashAggregate [codegen id : 14] +(57) HashAggregate [codegen id : 6] Input [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] Keys [3]: [ca_county#45, d_qoy#43, d_year#42] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] @@ -375,10 +375,10 @@ Results [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] Arguments: hashpartitioning(ca_county#45, d_qoy#43, d_year#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(59) CometColumnarToRow [codegen id : 15] +(59) CometNativeColumnarToRow Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -(60) HashAggregate [codegen id : 15] +(60) HashAggregate [codegen id : 7] Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] Keys [3]: [ca_county#45, d_qoy#43, d_year#42] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] @@ -389,7 +389,7 @@ Results [2]: [ca_county#45, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39) Input [2]: [ca_county#45, web_sales#49] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(62) BroadcastHashJoin [codegen id : 24] +(62) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#45] Join type: Inner @@ -403,40 +403,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 18] +(64) ColumnarToRow [codegen id : 8] Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -(65) Filter [codegen id : 18] +(65) Filter [codegen id : 8] Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] Condition : isnotnull(ws_bill_addr_sk#50) (66) ReusedExchange [Reuses operator id: 105] Output [3]: [d_date_sk#53, d_year#54, d_qoy#55] -(67) BroadcastHashJoin [codegen id : 18] +(67) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#52] Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None -(68) Project [codegen id : 18] +(68) Project [codegen id : 8] Output [4]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55] Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54, d_qoy#55] (69) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#56, ca_county#57] -(70) BroadcastHashJoin [codegen id : 18] +(70) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_bill_addr_sk#50] Right keys [1]: [ca_address_sk#56] Join type: Inner Join condition: None -(71) Project [codegen id : 18] +(71) Project [codegen id : 8] Output [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_address_sk#56, ca_county#57] -(72) HashAggregate [codegen id : 18] +(72) HashAggregate [codegen id : 8] Input [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] Keys [3]: [ca_county#57, d_qoy#55, d_year#54] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#51))] @@ -447,10 +447,10 @@ Results [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] Arguments: hashpartitioning(ca_county#57, d_qoy#55, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(74) CometColumnarToRow [codegen id : 19] +(74) CometNativeColumnarToRow Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -(75) HashAggregate [codegen id : 19] +(75) HashAggregate [codegen id : 9] Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] Keys [3]: [ca_county#57, d_qoy#55, d_year#54] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#51))] @@ -461,13 +461,13 @@ Results [2]: [ca_county#57, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#51) Input [2]: [ca_county#57, web_sales#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] -(77) BroadcastHashJoin [codegen id : 24] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#45] Right keys [1]: [ca_county#57] Join type: Inner Join condition: (CASE WHEN (web_sales#49 > 0.00) THEN (web_sales#60 / web_sales#49) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) -(78) Project [codegen id : 24] +(78) Project [codegen id : 12] Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60] Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, ca_county#57, web_sales#60] @@ -479,40 +479,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 22] +(80) ColumnarToRow [codegen id : 10] Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -(81) Filter [codegen id : 22] +(81) Filter [codegen id : 10] Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] Condition : isnotnull(ws_bill_addr_sk#61) (82) ReusedExchange [Reuses operator id: 109] Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] -(83) BroadcastHashJoin [codegen id : 22] +(83) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#63] Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(84) Project [codegen id : 22] +(84) Project [codegen id : 10] Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65, d_qoy#66] (85) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#67, ca_county#68] -(86) BroadcastHashJoin [codegen id : 22] +(86) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_bill_addr_sk#61] Right keys [1]: [ca_address_sk#67] Join type: Inner Join condition: None -(87) Project [codegen id : 22] +(87) Project [codegen id : 10] Output [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_address_sk#67, ca_county#68] -(88) HashAggregate [codegen id : 22] +(88) HashAggregate [codegen id : 10] Input [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] Keys [3]: [ca_county#68, d_qoy#66, d_year#65] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] @@ -523,10 +523,10 @@ Results [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] Arguments: hashpartitioning(ca_county#68, d_qoy#66, d_year#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(90) CometColumnarToRow [codegen id : 23] +(90) CometNativeColumnarToRow Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -(91) HashAggregate [codegen id : 23] +(91) HashAggregate [codegen id : 11] Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] Keys [3]: [ca_county#68, d_qoy#66, d_year#65] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] @@ -537,13 +537,13 @@ Results [2]: [ca_county#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62) Input [2]: [ca_county#68, web_sales#71] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] -(93) BroadcastHashJoin [codegen id : 24] +(93) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#45] Right keys [1]: [ca_county#68] Join type: Inner Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#71 / web_sales#60) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) -(94) Project [codegen id : 24] +(94) Project [codegen id : 12] Output [6]: [ca_county#9, d_year#6, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#75] Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60, ca_county#68, web_sales#71] @@ -555,14 +555,14 @@ Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENT Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75], [ca_county#9 ASC NULLS FIRST] -(97) CometColumnarToRow [codegen id : 25] +(97) CometNativeColumnarToRow Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (101) -+- * CometColumnarToRow (100) ++- CometNativeColumnarToRow (100) +- CometFilter (99) +- CometNativeScan parquet spark_catalog.default.date_dim (98) @@ -578,7 +578,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(100) CometColumnarToRow [codegen id : 1] +(100) CometNativeColumnarToRow Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (101) BroadcastExchange @@ -587,7 +587,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 BroadcastExchange (105) -+- * CometColumnarToRow (104) ++- CometNativeColumnarToRow (104) +- CometFilter (103) +- CometNativeScan parquet spark_catalog.default.date_dim (102) @@ -603,7 +603,7 @@ ReadSchema: struct Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) -(104) CometColumnarToRow [codegen id : 1] +(104) CometNativeColumnarToRow Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] (105) BroadcastExchange @@ -612,7 +612,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (109) -+- * CometColumnarToRow (108) ++- CometNativeColumnarToRow (108) +- CometFilter (107) +- CometNativeScan parquet spark_catalog.default.date_dim (106) @@ -628,7 +628,7 @@ ReadSchema: struct Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) -(108) CometColumnarToRow [codegen id : 1] +(108) CometNativeColumnarToRow Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] (109) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt index f6f3c8032a..03bcf0413e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project @@ -10,7 +10,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- BroadcastHashJoin : : : : :- HashAggregate - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- HashAggregate : : : : : +- Project @@ -22,20 +22,20 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : : +- BroadcastExchange : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project @@ -47,20 +47,20 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- BroadcastExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -72,20 +72,20 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -97,16 +97,16 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -118,16 +118,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -139,11 +139,11 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/simplified.txt index f3a0572081..0a36656045 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/simplified.txt @@ -1,22 +1,75 @@ -WholeStageCodegen (25) - 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 - WholeStageCodegen (24) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] +CometNativeColumnarToRow + 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 + WholeStageCodegen (12) + Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] + Project [ca_county,d_year,store_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow + BroadcastHashJoin [ca_county,ca_county] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #2 + WholeStageCodegen (1) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [ca_address_sk,ca_county] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #6 + WholeStageCodegen (2) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #7 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #9 + WholeStageCodegen (4) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,d_year,d_qoy,ca_county] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -26,130 +79,45 @@ WholeStageCodegen (25) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #10 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #10 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (7) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #12 + WholeStageCodegen (6) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) + BroadcastExchange #13 + WholeStageCodegen (9) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #14 + WholeStageCodegen (8) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,d_year,d_qoy,ca_county] BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] @@ -159,8 +127,30 @@ WholeStageCodegen (25) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 + ReusedExchange [d_date_sk,d_year,d_qoy] #7 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (11) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #16 + WholeStageCodegen (10) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #10 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt index 83d717e295..d300161c8f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt @@ -511,7 +511,7 @@ Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#6 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (94) -+- * CometColumnarToRow (93) ++- CometNativeColumnarToRow (93) +- CometFilter (92) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) CometColumnarToRow [codegen id : 1] +(93) CometNativeColumnarToRow Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometFilter (96) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * CometColumnarToRow (101) ++- CometNativeColumnarToRow (101) +- CometFilter (100) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(101) CometColumnarToRow [codegen id : 1] +(101) CometNativeColumnarToRow Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt index b4e7aa4db6..780504fb09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt index 2f97384320..745f9356ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt @@ -22,11 +22,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -45,11 +43,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -66,11 +62,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt index 83d717e295..d300161c8f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt @@ -511,7 +511,7 @@ Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#6 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (94) -+- * CometColumnarToRow (93) ++- CometNativeColumnarToRow (93) +- CometFilter (92) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) CometColumnarToRow [codegen id : 1] +(93) CometNativeColumnarToRow Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometFilter (96) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * CometColumnarToRow (101) ++- CometNativeColumnarToRow (101) +- CometFilter (100) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(101) CometColumnarToRow [codegen id : 1] +(101) CometNativeColumnarToRow Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt index b4e7aa4db6..780504fb09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange 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 2f97384320..745f9356ce 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 @@ -22,11 +22,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -45,11 +43,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -66,11 +62,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/explain.txt index df3987b0a0..5405030155 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * HashAggregate (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * Project (27) @@ -13,14 +13,14 @@ : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : +- BroadcastExchange (8) - : : +- * CometColumnarToRow (7) + : : +- CometNativeColumnarToRow (7) : : +- CometProject (6) : : +- CometFilter (5) : : +- CometNativeScan parquet spark_catalog.default.item (4) : +- BroadcastExchange (22) : +- * Filter (21) : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) + : +- CometNativeColumnarToRow (19) : +- CometColumnarExchange (18) : +- * HashAggregate (17) : +- * Project (16) @@ -40,10 +40,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 3] Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 3] Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) @@ -62,20 +62,20 @@ Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotn Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [i_item_sk#5] (8) BroadcastExchange Input [1]: [i_item_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 6] +(9) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 6] +(10) Project [codegen id : 3] Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] @@ -87,27 +87,27 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] +(12) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -(13) Filter [codegen id : 3] +(13) Filter [codegen id : 1] Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] Condition : isnotnull(cs_item_sk#7) (14) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#10] -(15) BroadcastHashJoin [codegen id : 3] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(16) Project [codegen id : 3] +(16) Project [codegen id : 1] Output [2]: [cs_item_sk#7, cs_ext_discount_amt#8] Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#10] -(17) HashAggregate [codegen id : 3] +(17) HashAggregate [codegen id : 1] Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] Keys [1]: [cs_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] @@ -118,17 +118,17 @@ Results [3]: [cs_item_sk#7, sum#13, count#14] Input [3]: [cs_item_sk#7, sum#13, count#14] Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) CometColumnarToRow [codegen id : 4] +(19) CometNativeColumnarToRow Input [3]: [cs_item_sk#7, sum#13, count#14] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [cs_item_sk#7, sum#13, count#14] Keys [1]: [cs_item_sk#7] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#15] Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -(21) Filter [codegen id : 4] +(21) Filter [codegen id : 2] Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) @@ -136,30 +136,30 @@ Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#5] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16) -(24) Project [codegen id : 6] +(24) Project [codegen id : 3] Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] (25) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#17] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 3] Output [1]: [cs_ext_discount_amt#2] Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#17] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 3] Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] @@ -170,10 +170,10 @@ Results [1]: [sum#19] Input [1]: [sum#19] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [1]: [sum#19] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 4] Input [1]: [sum#19] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] @@ -184,7 +184,7 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#20,17,2) AS Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -205,7 +205,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#2 Input [2]: [d_date_sk#17, d_date#22] Arguments: [d_date_sk#17], [d_date_sk#17] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt index 1eb8f397c3..f7096f5c3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -13,19 +13,19 @@ HashAggregate : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -35,12 +35,12 @@ HashAggregate : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/simplified.txt index 14c21b32a5..c9760ec371 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [cs_ext_discount_amt] [sum,sum] Project [cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -17,29 +17,25 @@ WholeStageCodegen (7) Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(cs_ext_discount_amt))] HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cs_item_sk] #5 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] Project [cs_item_sk,cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt index d6c10008ec..c4b57e9499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt @@ -175,7 +175,7 @@ Input [1]: [excess discount amount#18] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt index 943bb147ff..c7273327fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt index 07619fc999..78be4baea7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt index d6c10008ec..c4b57e9499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt @@ -175,7 +175,7 @@ Input [1]: [excess discount amount#18] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt index 943bb147ff..c7273327fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 07619fc999..78be4baea7 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 @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/explain.txt index a4e15583d4..632ce0e272 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (66) +- * HashAggregate (65) - +- * CometColumnarToRow (64) + +- CometNativeColumnarToRow (64) +- CometColumnarExchange (63) +- * HashAggregate (62) +- Union (61) :- * HashAggregate (28) - : +- * CometColumnarToRow (27) + : +- CometNativeColumnarToRow (27) : +- CometColumnarExchange (26) : +- * HashAggregate (25) : +- * Project (24) @@ -20,12 +20,12 @@ TakeOrderedAndProject (66) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometBroadcastHashJoin (20) : :- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.item (14) @@ -34,7 +34,7 @@ TakeOrderedAndProject (66) : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.item (16) :- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -50,7 +50,7 @@ TakeOrderedAndProject (66) : : +- ReusedExchange (35) : +- ReusedExchange (38) +- * HashAggregate (60) - +- * CometColumnarToRow (59) + +- CometNativeColumnarToRow (59) +- CometColumnarExchange (58) +- * HashAggregate (57) +- * Project (56) @@ -75,23 +75,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] @@ -110,20 +110,20 @@ Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isno Input [2]: [ca_address_sk#7, ca_gmt_offset#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [ca_address_sk#7] (11) BroadcastExchange Input [1]: [ca_address_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] @@ -162,24 +162,24 @@ Left output [2]: [i_item_sk#9, i_manufact_id#10] Right output [1]: [i_manufact_id#12] Arguments: [i_manufact_id#10], [i_manufact_id#12], LeftSemi, BuildRight -(21) CometColumnarToRow [codegen id : 3] +(21) CometNativeColumnarToRow Input [2]: [i_item_sk#9, i_manufact_id#10] (22) BroadcastExchange Input [2]: [i_item_sk#9, i_manufact_id#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 4] +(23) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#9] Join type: Inner Join condition: None -(24) Project [codegen id : 4] +(24) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#3, i_manufact_id#10] Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_manufact_id#10] -(25) HashAggregate [codegen id : 4] +(25) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_manufact_id#10] Keys [1]: [i_manufact_id#10] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -190,10 +190,10 @@ Results [2]: [i_manufact_id#10, sum#14] Input [2]: [i_manufact_id#10, sum#14] Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(27) CometColumnarToRow [codegen id : 5] +(27) CometNativeColumnarToRow Input [2]: [i_manufact_id#10, sum#14] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 2] Input [2]: [i_manufact_id#10, sum#14] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -208,53 +208,53 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 3] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -(31) Filter [codegen id : 9] +(31) Filter [codegen id : 3] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) (32) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#21] -(33) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#20] Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None -(34) Project [codegen id : 9] +(34) Project [codegen id : 3] Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#21] (35) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#22] -(36) BroadcastHashJoin [codegen id : 9] +(36) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_addr_sk#17] Right keys [1]: [ca_address_sk#22] Join type: Inner Join condition: None -(37) Project [codegen id : 9] +(37) Project [codegen id : 3] Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#22] (38) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#23, i_manufact_id#24] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#18] Right keys [1]: [i_item_sk#23] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 3] Output [2]: [cs_ext_sales_price#19, i_manufact_id#24] Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#23, i_manufact_id#24] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 3] Input [2]: [cs_ext_sales_price#19, i_manufact_id#24] Keys [1]: [i_manufact_id#24] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] @@ -265,10 +265,10 @@ Results [2]: [i_manufact_id#24, sum#26] Input [2]: [i_manufact_id#24, sum#26] Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [2]: [i_manufact_id#24, sum#26] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 4] Input [2]: [i_manufact_id#24, sum#26] Keys [1]: [i_manufact_id#24] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] @@ -283,53 +283,53 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 14] +(46) ColumnarToRow [codegen id : 5] Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] -(47) Filter [codegen id : 14] +(47) Filter [codegen id : 5] Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) (48) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#33] -(49) BroadcastHashJoin [codegen id : 14] +(49) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#32] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(50) Project [codegen id : 14] +(50) Project [codegen id : 5] Output [3]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31] Input [5]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32, d_date_sk#33] (51) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#34] -(52) BroadcastHashJoin [codegen id : 14] +(52) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_bill_addr_sk#30] Right keys [1]: [ca_address_sk#34] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 5] Output [2]: [ws_item_sk#29, ws_ext_sales_price#31] Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ca_address_sk#34] (54) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#35, i_manufact_id#36] -(55) BroadcastHashJoin [codegen id : 14] +(55) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#35] Join type: Inner Join condition: None -(56) Project [codegen id : 14] +(56) Project [codegen id : 5] Output [2]: [ws_ext_sales_price#31, i_manufact_id#36] Input [4]: [ws_item_sk#29, ws_ext_sales_price#31, i_item_sk#35, i_manufact_id#36] -(57) HashAggregate [codegen id : 14] +(57) HashAggregate [codegen id : 5] Input [2]: [ws_ext_sales_price#31, i_manufact_id#36] Keys [1]: [i_manufact_id#36] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#31))] @@ -340,10 +340,10 @@ Results [2]: [i_manufact_id#36, sum#38] Input [2]: [i_manufact_id#36, sum#38] Arguments: hashpartitioning(i_manufact_id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(59) CometColumnarToRow [codegen id : 15] +(59) CometNativeColumnarToRow Input [2]: [i_manufact_id#36, sum#38] -(60) HashAggregate [codegen id : 15] +(60) HashAggregate [codegen id : 6] Input [2]: [i_manufact_id#36, sum#38] Keys [1]: [i_manufact_id#36] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#31))] @@ -352,7 +352,7 @@ Results [2]: [i_manufact_id#36, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price (61) Union -(62) HashAggregate [codegen id : 16] +(62) HashAggregate [codegen id : 7] Input [2]: [i_manufact_id#10, total_sales#16] Keys [1]: [i_manufact_id#10] Functions [1]: [partial_sum(total_sales#16)] @@ -363,10 +363,10 @@ Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(64) CometColumnarToRow [codegen id : 17] +(64) CometNativeColumnarToRow Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -(65) HashAggregate [codegen id : 17] +(65) HashAggregate [codegen id : 8] Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(total_sales#16)] @@ -381,7 +381,7 @@ Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_manufact_id#10, total_sales Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometNativeScan parquet spark_catalog.default.date_dim (67) @@ -402,7 +402,7 @@ Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1 Input [3]: [d_date_sk#6, d_year#47, d_moy#48] Arguments: [d_date_sk#6], [d_date_sk#6] -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt index 79b2700a8c..38b3efb123 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,22 +19,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -43,7 +43,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -57,17 +57,17 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -76,7 +76,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -90,17 +90,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/simplified.txt index fd1c1d0ef2..edbc778e0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_manufact_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -27,40 +27,34 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #6 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (10) + CometNativeColumnarToRow + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [i_manufact_id] #6 + CometProject [i_manufact_id] + CometFilter [i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #7 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -79,12 +73,12 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ReusedExchange [ca_address_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (15) + WholeStageCodegen (6) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] BroadcastHashJoin [ws_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt index b7f7807f53..3fa1117b1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt @@ -346,7 +346,7 @@ Input [2]: [i_manufact_id#12, total_sales#41] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * CometColumnarToRow (65) ++- CometNativeColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) CometColumnarToRow [codegen id : 1] +(65) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt index 063fdb6faf..971bf3c1f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt index b4b84c2ac6..fc867ade9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt index b7f7807f53..3fa1117b1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt @@ -346,7 +346,7 @@ Input [2]: [i_manufact_id#12, total_sales#41] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (66) -+- * CometColumnarToRow (65) ++- CometNativeColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) CometColumnarToRow [codegen id : 1] +(65) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (66) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt index 063fdb6faf..971bf3c1f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 b4b84c2ac6..fc867ade9d 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/explain.txt index 7d332cb2e7..c14af34313 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometSort (34) +- CometColumnarExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) :- * Filter (25) : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -20,17 +20,17 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.customer (26) @@ -44,23 +44,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] @@ -79,20 +79,20 @@ Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND is Input [2]: [s_store_sk#8, s_county#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (11) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] @@ -111,24 +111,24 @@ Condition : ((((isnotnull(hd_vehicle_count#13) AND ((static_invoke(CharVarcharCo Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#10] (18) BroadcastExchange Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] @@ -139,17 +139,17 @@ Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) @@ -168,20 +168,20 @@ Condition : isnotnull(c_customer_sk#18) Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] (30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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] @@ -193,14 +193,14 @@ Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC 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: [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 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (40) -+- * CometColumnarToRow (39) ++- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometNativeScan parquet spark_catalog.default.date_dim (36) @@ -221,7 +221,7 @@ Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(39) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (40) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt index 9e4596c12c..40230582bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- BroadcastHashJoin :- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,27 +19,27 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/simplified.txt index 10af5ceadc..bbcdf458c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/simplified.txt @@ -1,59 +1,49 @@ -WholeStageCodegen (7) - 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 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] +CometNativeColumnarToRow + 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 + WholeStageCodegen (2) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/explain.txt index d8ee4f981e..bc110d3432 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/simplified.txt index 47d0e35dd2..3362a73ddd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 d8ee4f981e..bc110d3432 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 @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 47d0e35dd2..3362a73ddd 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/explain.txt index 7d1dadfda2..165be63b02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -13,7 +13,7 @@ TakeOrderedAndProject (45) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -35,12 +35,12 @@ TakeOrderedAndProject (45) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_address (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) @@ -57,7 +57,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] (6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] @@ -87,7 +87,7 @@ Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi @@ -100,19 +100,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] (13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] @@ -120,7 +120,7 @@ Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) @@ -133,19 +133,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] @@ -153,17 +153,17 @@ Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(25) Filter [codegen id : 4] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -182,20 +182,20 @@ Condition : isnotnull(ca_address_sk#16) Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) AS ca_state#18] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [2]: [ca_address_sk#16, ca_state#18] (31) BroadcastExchange Input [2]: [ca_address_sk#16, ca_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, ca_state#18] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#18] @@ -214,24 +214,24 @@ Condition : isnotnull(cd_demo_sk#19) Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#20, 1)) AS cd_gender#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] (38) BroadcastExchange Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 4] Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Input [8]: [c_current_cdemo_sk#4, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 4] Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [partial_count(1), partial_min(cd_dep_count#22), partial_max(cd_dep_count#22), partial_avg(cd_dep_count#22), partial_min(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_avg(cd_dep_employed_count#23), partial_min(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_avg(cd_dep_college_count#24)] @@ -242,10 +242,10 @@ Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 5] Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [count(1), min(cd_dep_count#22), max(cd_dep_count#22), avg(cd_dep_count#22), min(cd_dep_employed_count#23), max(cd_dep_employed_count#23), avg(cd_dep_employed_count#23), min(cd_dep_college_count#24), max(cd_dep_college_count#24), avg(cd_dep_college_count#24)] @@ -260,7 +260,7 @@ Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * CometColumnarToRow (49) ++- CometNativeColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometNativeScan parquet spark_catalog.default.date_dim (46) @@ -281,7 +281,7 @@ Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 2 Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt index 8935b6cc13..40a8ef4a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -22,12 +22,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,17 +49,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/simplified.txt index 48bc5637d0..a0b0348e6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -29,17 +29,15 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,7 +48,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -61,17 +59,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/explain.txt index 4db51f2231..76fcf0350b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#20) Input [2]: [ca_address_sk#20, ca_state#21] Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#20, ca_state#22] (33) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#22] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#23) Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/simplified.txt index f1fe09fb46..4acbd21997 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt index 4db51f2231..76fcf0350b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#20) Input [2]: [ca_address_sk#20, ca_state#21] Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#20, ca_state#22] (33) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#22] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#23) Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt index f1fe09fb46..4acbd21997 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/explain.txt index d3f2114c78..daac9ee68a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (31) +- * Project (30) +- Window (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometSort (27) +- CometColumnarExchange (26) +- * HashAggregate (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometColumnarExchange (23) +- * HashAggregate (22) +- * Expand (21) @@ -21,12 +21,12 @@ TakeOrderedAndProject (31) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometProject (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -40,23 +40,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (4) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] @@ -75,20 +75,20 @@ Condition : isnotnull(i_item_sk#8) Input [3]: [i_item_sk#8, i_class#9, i_category#10] Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#12] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#8, i_class#11, i_category#12] (11) BroadcastExchange Input [3]: [i_item_sk#8, i_class#11, i_category#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] @@ -107,28 +107,28 @@ Condition : ((isnotnull(s_state#14) AND (static_invoke(CharVarcharCodegenUtils.r Input [2]: [s_store_sk#13, s_state#14] Arguments: [s_store_sk#13], [s_store_sk#13] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [s_store_sk#13] (18) BroadcastExchange Input [1]: [s_store_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] -(21) Expand [codegen id : 4] +(21) Expand [codegen id : 1] Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 1] Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -139,10 +139,10 @@ Results [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] Arguments: hashpartitioning(i_category#15, i_class#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 5] +(24) CometNativeColumnarToRow Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 2] Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] @@ -157,14 +157,14 @@ Arguments: hashpartitioning(_w1#27, _w2#28, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] Arguments: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28], [_w1#27 ASC NULLS FIRST, _w2#28 ASC NULLS FIRST, _w0#26 ASC NULLS FIRST] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] (29) Window Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] Arguments: [rank(_w0#26) windowspecdefinition(_w1#27, _w2#28, _w0#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#29], [_w1#27, _w2#28], [_w0#26 ASC NULLS FIRST] -(30) Project [codegen id : 7] +(30) Project [codegen id : 3] Output [5]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] Input [8]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28, rank_within_parent#29] @@ -176,7 +176,7 @@ Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#30) AND (d_year#30 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#7, d_year#30] Arguments: [d_date_sk#7], [d_date_sk#7] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt index bd826eb3a3..aa40ed68cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -20,22 +20,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/simplified.txt index d6f8548c89..a6427d7111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/simplified.txt @@ -1,54 +1,46 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) + WholeStageCodegen (3) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] + Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt index 634c01506e..dddd6648e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt @@ -172,7 +172,7 @@ Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -193,7 +193,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt index 89c523a388..2bf9a90726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt index 61782e3011..248d0bc6e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt index 634c01506e..dddd6648e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt @@ -172,7 +172,7 @@ Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -193,7 +193,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt index 89c523a388..2bf9a90726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 61782e3011..248d0bc6e8 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 @@ -22,12 +22,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/explain.txt index 7fcb5f8109..ec36819499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (26) +CometNativeColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometColumnarExchange (23) @@ -11,7 +11,7 @@ : +- * BroadcastHashJoin Inner BuildRight (13) : :- * Project (11) : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -21,7 +21,7 @@ : : +- * ColumnarToRow (6) : : +- Scan parquet spark_catalog.default.inventory (5) : +- ReusedExchange (12) - +- * CometColumnarToRow (19) + +- CometNativeColumnarToRow (19) +- CometProject (18) +- CometFilter (17) +- CometNativeScan parquet spark_catalog.default.catalog_sales (16) @@ -42,7 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#2, 16)) AS i_item_id#6, i_item_desc#3, i_current_price#4] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] (5) Scan parquet spark_catalog.default.inventory @@ -68,26 +68,26 @@ Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] Input [2]: [inv_item_sk#7, inv_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [inv_item_sk#7] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] (12) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#11] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] @@ -110,20 +110,20 @@ Condition : isnotnull(cs_item_sk#12) Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] Arguments: [cs_item_sk#12], [cs_item_sk#12] -(19) CometColumnarToRow +(19) CometNativeColumnarToRow Input [1]: [cs_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#12] Join type: Inner Join condition: None -(21) Project [codegen id : 4] +(21) Project [codegen id : 3] Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#12] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 3] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Functions: [] @@ -143,14 +143,14 @@ Functions: [] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] -(26) CometColumnarToRow [codegen id : 5] +(26) CometNativeColumnarToRow Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.date_dim (27) @@ -171,7 +171,7 @@ Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#14] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt index fb7dbb8d72..296b2c8117 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -10,7 +10,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -21,16 +21,16 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_sales diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/simplified.txt index 88a3895990..3f16bc9b50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/simplified.txt @@ -1,45 +1,41 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,cs_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [inv_item_sk,inv_date_sk] + Filter [inv_quantity_on_hand,inv_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + CometNativeColumnarToRow + CometProject [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/explain.txt index ff90c33aff..c37f8fcfab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt index abe0937d1d..be3d6196b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/simplified.txt index 7ab646a628..f5ddbb2d04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt index ff90c33aff..c37f8fcfab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/extended.txt index abe0937d1d..be3d6196b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt index 7ab646a628..f5ddbb2d04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/explain.txt index 89b70107e2..542b99cdee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (49) +CometNativeColumnarToRow (49) +- CometHashAggregate (48) +- CometExchange (47) +- CometHashAggregate (46) @@ -18,7 +18,7 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer (7) @@ -58,23 +58,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) (4) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#4, d_date#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#2] Right keys [1]: [d_date_sk#4] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, d_date#5] Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] @@ -93,24 +93,24 @@ Condition : isnotnull(c_customer_sk#6) Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#7, 20)) AS c_first_name#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#8, 30)) AS c_last_name#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] (11) BroadcastExchange Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [c_last_name#10, c_first_name#9, d_date#5] Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [c_last_name#10, c_first_name#9, d_date#5] Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] Functions: [] @@ -134,40 +134,40 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Condition : isnotnull(cs_bill_customer_sk#11) (20) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#13, d_date#14] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [2]: [cs_bill_customer_sk#11, d_date#14] Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] (23) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_bill_customer_sk#11] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [3]: [c_last_name#17, c_first_name#16, d_date#14] Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] @@ -200,40 +200,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 9] +(32) ColumnarToRow [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(33) Filter [codegen id : 9] +(33) Filter [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Condition : isnotnull(ws_bill_customer_sk#18) (34) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#20, d_date#21] -(35) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(36) Project [codegen id : 9] +(36) Project [codegen id : 3] Output [2]: [ws_bill_customer_sk#18, d_date#21] Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] (37) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(38) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_bill_customer_sk#18] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(39) Project [codegen id : 3] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 3] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] @@ -275,14 +275,14 @@ Input [1]: [count#25] Keys: [] Functions [1]: [count(1)] -(49) CometColumnarToRow [codegen id : 10] +(49) CometNativeColumnarToRow Input [1]: [count(1)#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometNativeScan parquet spark_catalog.default.date_dim (50) @@ -303,7 +303,7 @@ Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_mo Input [3]: [d_date_sk#4, d_date#5, d_month_seq#27] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt index 061475900e..6d9f5c7a17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -17,17 +17,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -44,12 +44,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer @@ -66,12 +66,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/simplified.txt index 547965d09c..f127e42be4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/simplified.txt @@ -1,75 +1,69 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] +CometNativeColumnarToRow + CometHashAggregate [count] [count(1),count(1)] + CometExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (1) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #5 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange [c_last_name,c_first_name,d_date] #5 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + CometBroadcastExchange [c_last_name,c_first_name,d_date] #7 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/explain.txt index 3079061df7..f503edc37a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/explain.txt @@ -273,7 +273,7 @@ Input [1]: [count(1)#29] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -294,7 +294,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt index 55af64efc6..715ff39db7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/simplified.txt index 7ceccf5d2c..6c90e8467d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt index 3079061df7..f503edc37a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt @@ -273,7 +273,7 @@ Input [1]: [count(1)#29] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -294,7 +294,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/extended.txt index 55af64efc6..715ff39db7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 7ceccf5d2c..6c90e8467d 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/explain.txt index 540361017b..e7b2b342c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- * BroadcastHashJoin Inner BuildRight (44) :- * Project (24) : +- * Filter (23) : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometColumnarExchange (20) : +- * HashAggregate (19) : +- * Project (18) @@ -19,11 +19,11 @@ : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.inventory (1) : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) + : : : +- CometNativeColumnarToRow (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) + : : +- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : +- ReusedExchange (16) @@ -31,7 +31,7 @@ +- * Project (42) +- * Filter (41) +- * HashAggregate (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometColumnarExchange (38) +- * HashAggregate (37) +- * Project (36) @@ -56,10 +56,10 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) @@ -74,20 +74,20 @@ ReadSchema: struct Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [i_item_sk#6] (7) BroadcastExchange Input [1]: [i_item_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] @@ -102,37 +102,37 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] (13) BroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] (16) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#9, d_moy#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] @@ -143,21 +143,21 @@ Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometColumnarToRow [codegen id : 10] +(21) CometNativeColumnarToRow Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(22) HashAggregate [codegen id : 10] +(22) HashAggregate [codegen id : 4] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(23) Filter [codegen id : 10] +(23) Filter [codegen id : 4] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END -(24) Project [codegen id : 10] +(24) Project [codegen id : 4] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -169,53 +169,53 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 8] +(26) ColumnarToRow [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(27) Filter [codegen id : 8] +(27) Filter [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) (28) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(29) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(30) Project [codegen id : 8] +(30) Project [codegen id : 2] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] (31) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] (34) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#34, d_moy#35] -(35) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(36) Project [codegen id : 8] +(36) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(37) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 2] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] @@ -226,21 +226,21 @@ Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n# Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) CometColumnarToRow [codegen id : 9] +(39) CometNativeColumnarToRow Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 3] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] -(41) Filter [codegen id : 9] +(41) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#47)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#46 / knownfloatingpointnormalized(normalizenanandzero(mean#47))))) > 1.0) END -(42) Project [codegen id : 9] +(42) Project [codegen id : 3] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#47)) = 0.0) THEN null ELSE (stdev#46 / knownfloatingpointnormalized(normalizenanandzero(mean#47))) END AS cov#48] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] @@ -248,7 +248,7 @@ Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(44) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner @@ -262,14 +262,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 11] +(47) CometNativeColumnarToRow Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometNativeScan parquet spark_catalog.default.date_dim (48) @@ -290,7 +290,7 @@ Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2 Input [3]: [d_date_sk#9, d_year#49, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#10] (52) BroadcastExchange @@ -299,7 +299,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -320,7 +320,7 @@ Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2 Input [3]: [d_date_sk#34, d_year#50, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#34, d_moy#35] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt index e44cca98a1..d9cec17b28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- BroadcastHashJoin :- Project : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,20 +19,20 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,7 +40,7 @@ CometColumnarToRow +- Project +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -54,20 +54,20 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/simplified.txt index 11dd59a97f..ff7f9077d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/simplified.txt @@ -1,84 +1,74 @@ -WholeStageCodegen (11) - 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 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + 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 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (1) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (2) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk] #4 + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + ReusedExchange [i_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/explain.txt index 4e7b77d7ec..779490dbe2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/explain.txt @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/simplified.txt index e54435ae10..618e6d67db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 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 4e7b77d7ec..779490dbe2 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 @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e54435ae10..618e6d67db 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/explain.txt index 1e019e0e20..2c259861d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- * BroadcastHashJoin Inner BuildRight (44) :- * Project (24) : +- * Filter (23) : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometColumnarExchange (20) : +- * HashAggregate (19) : +- * Project (18) @@ -19,11 +19,11 @@ : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.inventory (1) : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) + : : : +- CometNativeColumnarToRow (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) + : : +- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : +- ReusedExchange (16) @@ -31,7 +31,7 @@ +- * Project (42) +- * Filter (41) +- * HashAggregate (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometColumnarExchange (38) +- * HashAggregate (37) +- * Project (36) @@ -56,10 +56,10 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) @@ -74,20 +74,20 @@ ReadSchema: struct Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [i_item_sk#6] (7) BroadcastExchange Input [1]: [i_item_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] @@ -102,37 +102,37 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] (13) BroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] (16) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#9, d_moy#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] @@ -143,21 +143,21 @@ Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometColumnarToRow [codegen id : 10] +(21) CometNativeColumnarToRow Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(22) HashAggregate [codegen id : 10] +(22) HashAggregate [codegen id : 4] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(23) Filter [codegen id : 10] +(23) Filter [codegen id : 4] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.5) END) -(24) Project [codegen id : 10] +(24) Project [codegen id : 4] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -169,53 +169,53 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 8] +(26) ColumnarToRow [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(27) Filter [codegen id : 8] +(27) Filter [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) (28) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(29) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(30) Project [codegen id : 8] +(30) Project [codegen id : 2] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] (31) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] (34) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#34, d_moy#35] -(35) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(36) Project [codegen id : 8] +(36) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(37) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 2] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] @@ -226,21 +226,21 @@ Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n# Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) CometColumnarToRow [codegen id : 9] +(39) CometNativeColumnarToRow Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 3] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] -(41) Filter [codegen id : 9] +(41) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#47)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#46 / knownfloatingpointnormalized(normalizenanandzero(mean#47))))) > 1.0) END -(42) Project [codegen id : 9] +(42) Project [codegen id : 3] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#47)) = 0.0) THEN null ELSE (stdev#46 / knownfloatingpointnormalized(normalizenanandzero(mean#47))) END AS cov#48] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] @@ -248,7 +248,7 @@ Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(44) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner @@ -262,14 +262,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 11] +(47) CometNativeColumnarToRow Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometNativeScan parquet spark_catalog.default.date_dim (48) @@ -290,7 +290,7 @@ Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2 Input [3]: [d_date_sk#9, d_year#49, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#10] (52) BroadcastExchange @@ -299,7 +299,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -320,7 +320,7 @@ Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2 Input [3]: [d_date_sk#34, d_year#50, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#34, d_moy#35] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt index e44cca98a1..d9cec17b28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- BroadcastHashJoin :- Project : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,20 +19,20 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,7 +40,7 @@ CometColumnarToRow +- Project +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -54,20 +54,20 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/simplified.txt index 11dd59a97f..ff7f9077d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/simplified.txt @@ -1,84 +1,74 @@ -WholeStageCodegen (11) - 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 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + 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 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (1) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (2) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk] #4 + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + ReusedExchange [i_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/explain.txt index 7d8df40ba0..9c05b59a06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/explain.txt @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/simplified.txt index e54435ae10..618e6d67db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 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 7d8df40ba0..9c05b59a06 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 @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e54435ae10..618e6d67db 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/explain.txt index 57160be074..46b78f789f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/explain.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject (120) : : : :- * BroadcastHashJoin Inner BuildRight (37) : : : : :- * Filter (18) : : : : : +- * HashAggregate (17) - : : : : : +- * CometColumnarToRow (16) + : : : : : +- CometNativeColumnarToRow (16) : : : : : +- CometColumnarExchange (15) : : : : : +- * HashAggregate (14) : : : : : +- * Project (13) : : : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : : : :- * Project (10) : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * CometColumnarToRow (4) + : : : : : : :- CometNativeColumnarToRow (4) : : : : : : : +- CometProject (3) : : : : : : : +- CometFilter (2) : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -29,14 +29,14 @@ TakeOrderedAndProject (120) : : : : : +- ReusedExchange (11) : : : : +- BroadcastExchange (36) : : : : +- * HashAggregate (35) - : : : : +- * CometColumnarToRow (34) + : : : : +- CometNativeColumnarToRow (34) : : : : +- CometColumnarExchange (33) : : : : +- * HashAggregate (32) : : : : +- * Project (31) : : : : +- * BroadcastHashJoin Inner BuildRight (30) : : : : :- * Project (28) : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * CometColumnarToRow (22) + : : : : : :- CometNativeColumnarToRow (22) : : : : : : +- CometProject (21) : : : : : : +- CometFilter (20) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -48,14 +48,14 @@ TakeOrderedAndProject (120) : : : +- BroadcastExchange (56) : : : +- * Filter (55) : : : +- * HashAggregate (54) - : : : +- * CometColumnarToRow (53) + : : : +- CometNativeColumnarToRow (53) : : : +- CometColumnarExchange (52) : : : +- * HashAggregate (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) : : : :- * Project (47) : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * CometColumnarToRow (41) + : : : : :- CometNativeColumnarToRow (41) : : : : : +- CometProject (40) : : : : : +- CometFilter (39) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -66,14 +66,14 @@ TakeOrderedAndProject (120) : : : +- ReusedExchange (48) : : +- BroadcastExchange (76) : : +- * HashAggregate (75) - : : +- * CometColumnarToRow (74) + : : +- CometNativeColumnarToRow (74) : : +- CometColumnarExchange (73) : : +- * HashAggregate (72) : : +- * Project (71) : : +- * BroadcastHashJoin Inner BuildRight (70) : : :- * Project (68) : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : :- * CometColumnarToRow (62) + : : : :- CometNativeColumnarToRow (62) : : : : +- CometProject (61) : : : : +- CometFilter (60) : : : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -85,14 +85,14 @@ TakeOrderedAndProject (120) : +- BroadcastExchange (97) : +- * Filter (96) : +- * HashAggregate (95) - : +- * CometColumnarToRow (94) + : +- CometNativeColumnarToRow (94) : +- CometColumnarExchange (93) : +- * HashAggregate (92) : +- * Project (91) : +- * BroadcastHashJoin Inner BuildRight (90) : :- * Project (88) : : +- * BroadcastHashJoin Inner BuildRight (87) - : : :- * CometColumnarToRow (82) + : : :- CometNativeColumnarToRow (82) : : : +- CometProject (81) : : : +- CometFilter (80) : : : +- CometNativeScan parquet spark_catalog.default.customer (79) @@ -103,14 +103,14 @@ TakeOrderedAndProject (120) : +- ReusedExchange (89) +- BroadcastExchange (117) +- * HashAggregate (116) - +- * CometColumnarToRow (115) + +- CometNativeColumnarToRow (115) +- CometColumnarExchange (114) +- * HashAggregate (113) +- * Project (112) +- * BroadcastHashJoin Inner BuildRight (111) :- * Project (109) : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * CometColumnarToRow (103) + : :- CometNativeColumnarToRow (103) : : +- CometProject (102) : : +- CometFilter (101) : : +- CometNativeScan parquet spark_catalog.default.customer (100) @@ -136,7 +136,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharC Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] (5) Scan parquet spark_catalog.default.store_sales @@ -158,30 +158,30 @@ Condition : isnotnull(ss_customer_sk#15) Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#15] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] (11) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#22, d_year#23] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#20] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] @@ -192,17 +192,17 @@ Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cus Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 24] +(16) CometNativeColumnarToRow Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] -(17) HashAggregate [codegen id : 24] +(17) HashAggregate [codegen id : 18] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] Aggregate Attributes [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28] Results [2]: [c_customer_id#9 AS customer_id#29, sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28 AS year_total#30] -(18) Filter [codegen id : 24] +(18) Filter [codegen id : 18] Input [2]: [customer_id#29, year_total#30] Condition : (isnotnull(year_total#30) AND (year_total#30 > 0.000000)) @@ -221,7 +221,7 @@ Condition : (isnotnull(c_customer_sk#31) AND isnotnull(static_invoke(CharVarchar Input [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] Arguments: [c_customer_sk#31, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44], [c_customer_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#32, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#35, 1)) AS c_preferred_cust_flag#42, c_birth_country#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#37, 13)) AS c_login#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#38, 50)) AS c_email_address#44] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [8]: [c_customer_sk#31, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44] (23) Scan parquet spark_catalog.default.store_sales @@ -232,10 +232,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#50), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [6]: [ss_customer_sk#45, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [6]: [ss_customer_sk#45, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] Condition : isnotnull(ss_customer_sk#45) @@ -243,30 +243,30 @@ Condition : isnotnull(ss_customer_sk#45) Input [6]: [ss_customer_sk#45, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#31] Right keys [1]: [ss_customer_sk#45] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [12]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] Input [14]: [c_customer_sk#31, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_customer_sk#45, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] (29) ReusedExchange [Reuses operator id: 128] Output [2]: [d_date_sk#52, d_year#53] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#50] Right keys [1]: [d_date_sk#52] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [12]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, d_year#53] Input [14]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50, d_date_sk#52, d_year#53] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [12]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, d_year#53] Keys [8]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53] Functions [1]: [partial_sum(((((ss_ext_list_price#49 - ss_ext_wholesale_cost#48) - ss_ext_discount_amt#46) + ss_ext_sales_price#47) / 2))] @@ -277,10 +277,10 @@ Results [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cu Input [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53, sum#56, isEmpty#57] Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53, sum#56, isEmpty#57] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53, sum#56, isEmpty#57] Keys [8]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53] Functions [1]: [sum(((((ss_ext_list_price#49 - ss_ext_wholesale_cost#48) - ss_ext_discount_amt#46) + ss_ext_sales_price#47) / 2))] @@ -291,7 +291,7 @@ Results [8]: [c_customer_id#39 AS customer_id#58, c_first_name#40 AS customer_fi Input [8]: [customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#65] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 24] +(37) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#58] Join type: Inner @@ -312,7 +312,7 @@ Condition : (isnotnull(c_customer_sk#66) AND isnotnull(static_invoke(CharVarchar Input [8]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73] Arguments: [c_customer_sk#66, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79], [c_customer_sk#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#67, 16)) AS c_customer_id#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#68, 20)) AS c_first_name#75, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#69, 30)) AS c_last_name#76, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#70, 1)) AS c_preferred_cust_flag#77, c_birth_country#71, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#72, 13)) AS c_login#78, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#73, 50)) AS c_email_address#79] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [8]: [c_customer_sk#66, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79] (42) Scan parquet spark_catalog.default.catalog_sales @@ -323,10 +323,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#85), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Condition : isnotnull(cs_bill_customer_sk#80) @@ -334,30 +334,30 @@ Condition : isnotnull(cs_bill_customer_sk#80) Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#66] Right keys [1]: [cs_bill_customer_sk#80] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] Input [14]: [c_customer_sk#66, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] (48) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#86, d_year#87] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#85] Right keys [1]: [d_date_sk#86] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#87] Input [14]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85, d_date_sk#86, d_year#87] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#87] Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87] Functions [1]: [partial_sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))] @@ -368,17 +368,17 @@ Results [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cu Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87, sum#90, isEmpty#91] Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87, sum#90, isEmpty#91] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87, sum#90, isEmpty#91] Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87] Functions [1]: [sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))] Aggregate Attributes [1]: [sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))#92] Results [2]: [c_customer_id#74 AS customer_id#93, sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))#92 AS year_total#94] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#93, year_total#94] Condition : (isnotnull(year_total#94) AND (year_total#94 > 0.000000)) @@ -386,13 +386,13 @@ Condition : (isnotnull(year_total#94) AND (year_total#94 > 0.000000)) Input [2]: [customer_id#93, year_total#94] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 24] +(57) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#93] Join type: Inner Join condition: None -(58) Project [codegen id : 24] +(58) Project [codegen id : 18] Output [11]: [customer_id#29, year_total#30, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#65, year_total#94] Input [12]: [customer_id#29, year_total#30, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#65, customer_id#93, year_total#94] @@ -411,7 +411,7 @@ Condition : (isnotnull(c_customer_sk#95) AND isnotnull(static_invoke(CharVarchar Input [8]: [c_customer_sk#95, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#100, c_login#101, c_email_address#102] Arguments: [c_customer_sk#95, c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108], [c_customer_sk#95, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#96, 16)) AS c_customer_id#103, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#97, 20)) AS c_first_name#104, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#98, 30)) AS c_last_name#105, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#99, 1)) AS c_preferred_cust_flag#106, c_birth_country#100, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#101, 13)) AS c_login#107, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#102, 50)) AS c_email_address#108] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [8]: [c_customer_sk#95, c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108] (63) Scan parquet spark_catalog.default.catalog_sales @@ -422,10 +422,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#114), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [6]: [cs_bill_customer_sk#109, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [6]: [cs_bill_customer_sk#109, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] Condition : isnotnull(cs_bill_customer_sk#109) @@ -433,30 +433,30 @@ Condition : isnotnull(cs_bill_customer_sk#109) Input [6]: [cs_bill_customer_sk#109, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#95] Right keys [1]: [cs_bill_customer_sk#109] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [12]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] Input [14]: [c_customer_sk#95, c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_bill_customer_sk#109, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] (69) ReusedExchange [Reuses operator id: 128] Output [2]: [d_date_sk#115, d_year#116] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#114] Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [12]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, d_year#116] Input [14]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114, d_date_sk#115, d_year#116] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [12]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, d_year#116] Keys [8]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116] Functions [1]: [partial_sum(((((cs_ext_list_price#113 - cs_ext_wholesale_cost#112) - cs_ext_discount_amt#110) + cs_ext_sales_price#111) / 2))] @@ -467,10 +467,10 @@ Results [10]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred Input [10]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116, sum#119, isEmpty#120] Arguments: hashpartitioning(c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [10]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116, sum#119, isEmpty#120] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [10]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116, sum#119, isEmpty#120] Keys [8]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116] Functions [1]: [sum(((((cs_ext_list_price#113 - cs_ext_wholesale_cost#112) - cs_ext_discount_amt#110) + cs_ext_sales_price#111) / 2))] @@ -481,13 +481,13 @@ Results [2]: [c_customer_id#103 AS customer_id#121, sum(((((cs_ext_list_price#11 Input [2]: [customer_id#121, year_total#122] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 24] +(77) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#121] Join type: Inner Join condition: (CASE WHEN (year_total#94 > 0.000000) THEN (year_total#122 / year_total#94) END > CASE WHEN (year_total#30 > 0.000000) THEN (year_total#65 / year_total#30) END) -(78) Project [codegen id : 24] +(78) Project [codegen id : 18] Output [10]: [customer_id#29, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#94, year_total#122] Input [13]: [customer_id#29, year_total#30, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#65, year_total#94, customer_id#121, year_total#122] @@ -506,7 +506,7 @@ Condition : (isnotnull(c_customer_sk#123) AND isnotnull(static_invoke(CharVarcha Input [8]: [c_customer_sk#123, c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130] Arguments: [c_customer_sk#123, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136], [c_customer_sk#123, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#124, 16)) AS c_customer_id#131, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#125, 20)) AS c_first_name#132, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#126, 30)) AS c_last_name#133, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#127, 1)) AS c_preferred_cust_flag#134, c_birth_country#128, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#129, 13)) AS c_login#135, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#130, 50)) AS c_email_address#136] -(82) CometColumnarToRow [codegen id : 18] +(82) CometNativeColumnarToRow Input [8]: [c_customer_sk#123, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136] (83) Scan parquet spark_catalog.default.web_sales @@ -517,10 +517,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#142), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(84) ColumnarToRow [codegen id : 16] +(84) ColumnarToRow [codegen id : 12] Input [6]: [ws_bill_customer_sk#137, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] -(85) Filter [codegen id : 16] +(85) Filter [codegen id : 12] Input [6]: [ws_bill_customer_sk#137, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] Condition : isnotnull(ws_bill_customer_sk#137) @@ -528,30 +528,30 @@ Condition : isnotnull(ws_bill_customer_sk#137) Input [6]: [ws_bill_customer_sk#137, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(87) BroadcastHashJoin [codegen id : 18] +(87) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_customer_sk#123] Right keys [1]: [ws_bill_customer_sk#137] Join type: Inner Join condition: None -(88) Project [codegen id : 18] +(88) Project [codegen id : 13] Output [12]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] Input [14]: [c_customer_sk#123, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_bill_customer_sk#137, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] (89) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#143, d_year#144] -(90) BroadcastHashJoin [codegen id : 18] +(90) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#142] Right keys [1]: [d_date_sk#143] Join type: Inner Join condition: None -(91) Project [codegen id : 18] +(91) Project [codegen id : 13] Output [12]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, d_year#144] Input [14]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142, d_date_sk#143, d_year#144] -(92) HashAggregate [codegen id : 18] +(92) HashAggregate [codegen id : 13] Input [12]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, d_year#144] Keys [8]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144] Functions [1]: [partial_sum(((((ws_ext_list_price#141 - ws_ext_wholesale_cost#140) - ws_ext_discount_amt#138) + ws_ext_sales_price#139) / 2))] @@ -562,17 +562,17 @@ Results [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred Input [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144, sum#147, isEmpty#148] Arguments: hashpartitioning(c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(94) CometColumnarToRow [codegen id : 19] +(94) CometNativeColumnarToRow Input [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144, sum#147, isEmpty#148] -(95) HashAggregate [codegen id : 19] +(95) HashAggregate [codegen id : 14] Input [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144, sum#147, isEmpty#148] Keys [8]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144] Functions [1]: [sum(((((ws_ext_list_price#141 - ws_ext_wholesale_cost#140) - ws_ext_discount_amt#138) + ws_ext_sales_price#139) / 2))] Aggregate Attributes [1]: [sum(((((ws_ext_list_price#141 - ws_ext_wholesale_cost#140) - ws_ext_discount_amt#138) + ws_ext_sales_price#139) / 2))#149] Results [2]: [c_customer_id#131 AS customer_id#150, sum(((((ws_ext_list_price#141 - ws_ext_wholesale_cost#140) - ws_ext_discount_amt#138) + ws_ext_sales_price#139) / 2))#149 AS year_total#151] -(96) Filter [codegen id : 19] +(96) Filter [codegen id : 14] Input [2]: [customer_id#150, year_total#151] Condition : (isnotnull(year_total#151) AND (year_total#151 > 0.000000)) @@ -580,13 +580,13 @@ Condition : (isnotnull(year_total#151) AND (year_total#151 > 0.000000)) Input [2]: [customer_id#150, year_total#151] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] -(98) BroadcastHashJoin [codegen id : 24] +(98) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#150] Join type: Inner Join condition: None -(99) Project [codegen id : 24] +(99) Project [codegen id : 18] Output [11]: [customer_id#29, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#94, year_total#122, year_total#151] Input [12]: [customer_id#29, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#94, year_total#122, customer_id#150, year_total#151] @@ -605,7 +605,7 @@ Condition : (isnotnull(c_customer_sk#152) AND isnotnull(static_invoke(CharVarcha Input [8]: [c_customer_sk#152, c_customer_id#153, c_first_name#154, c_last_name#155, c_preferred_cust_flag#156, c_birth_country#157, c_login#158, c_email_address#159] Arguments: [c_customer_sk#152, c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165], [c_customer_sk#152, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#153, 16)) AS c_customer_id#160, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#154, 20)) AS c_first_name#161, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#155, 30)) AS c_last_name#162, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#156, 1)) AS c_preferred_cust_flag#163, c_birth_country#157, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#158, 13)) AS c_login#164, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#159, 50)) AS c_email_address#165] -(103) CometColumnarToRow [codegen id : 22] +(103) CometNativeColumnarToRow Input [8]: [c_customer_sk#152, c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165] (104) Scan parquet spark_catalog.default.web_sales @@ -616,10 +616,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#171), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 20] +(105) ColumnarToRow [codegen id : 15] Input [6]: [ws_bill_customer_sk#166, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] -(106) Filter [codegen id : 20] +(106) Filter [codegen id : 15] Input [6]: [ws_bill_customer_sk#166, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] Condition : isnotnull(ws_bill_customer_sk#166) @@ -627,30 +627,30 @@ Condition : isnotnull(ws_bill_customer_sk#166) Input [6]: [ws_bill_customer_sk#166, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -(108) BroadcastHashJoin [codegen id : 22] +(108) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_customer_sk#152] Right keys [1]: [ws_bill_customer_sk#166] Join type: Inner Join condition: None -(109) Project [codegen id : 22] +(109) Project [codegen id : 16] Output [12]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] Input [14]: [c_customer_sk#152, c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_bill_customer_sk#166, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] (110) ReusedExchange [Reuses operator id: 128] Output [2]: [d_date_sk#172, d_year#173] -(111) BroadcastHashJoin [codegen id : 22] +(111) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_sold_date_sk#171] Right keys [1]: [d_date_sk#172] Join type: Inner Join condition: None -(112) Project [codegen id : 22] +(112) Project [codegen id : 16] Output [12]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, d_year#173] Input [14]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171, d_date_sk#172, d_year#173] -(113) HashAggregate [codegen id : 22] +(113) HashAggregate [codegen id : 16] Input [12]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, d_year#173] Keys [8]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173] Functions [1]: [partial_sum(((((ws_ext_list_price#170 - ws_ext_wholesale_cost#169) - ws_ext_discount_amt#167) + ws_ext_sales_price#168) / 2))] @@ -661,10 +661,10 @@ Results [10]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred Input [10]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173, sum#176, isEmpty#177] Arguments: hashpartitioning(c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(115) CometColumnarToRow [codegen id : 23] +(115) CometNativeColumnarToRow Input [10]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173, sum#176, isEmpty#177] -(116) HashAggregate [codegen id : 23] +(116) HashAggregate [codegen id : 17] Input [10]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173, sum#176, isEmpty#177] Keys [8]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173] Functions [1]: [sum(((((ws_ext_list_price#170 - ws_ext_wholesale_cost#169) - ws_ext_discount_amt#167) + ws_ext_sales_price#168) / 2))] @@ -675,13 +675,13 @@ Results [2]: [c_customer_id#160 AS customer_id#178, sum(((((ws_ext_list_price#17 Input [2]: [customer_id#178, year_total#179] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] -(118) BroadcastHashJoin [codegen id : 24] +(118) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#178] Join type: Inner Join condition: (CASE WHEN (year_total#94 > 0.000000) THEN (year_total#122 / year_total#94) END > CASE WHEN (year_total#151 > 0.000000) THEN (year_total#179 / year_total#151) END) -(119) Project [codegen id : 24] +(119) Project [codegen id : 18] Output [7]: [customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64] Input [13]: [customer_id#29, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#94, year_total#122, year_total#151, customer_id#178, year_total#179] @@ -693,7 +693,7 @@ Arguments: 100, [customer_id#58 ASC NULLS FIRST, customer_first_name#59 ASC NULL Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometFilter (122) +- CometNativeScan parquet spark_catalog.default.date_dim (121) @@ -709,7 +709,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [2]: [d_date_sk#22, d_year#23] (124) BroadcastExchange @@ -718,7 +718,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#50 IN dynamicpruning#51 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometNativeScan parquet spark_catalog.default.date_dim (125) @@ -734,7 +734,7 @@ ReadSchema: struct Input [2]: [d_date_sk#52, d_year#53] Condition : ((isnotnull(d_year#53) AND (d_year#53 = 2002)) AND isnotnull(d_date_sk#52)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#52, d_year#53] (128) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt index 54e1a1c11a..737217ff91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt @@ -10,14 +10,14 @@ TakeOrderedAndProject : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- HashAggregate - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- HashAggregate : : : : : +- Project : : : : : +- BroadcastHashJoin : : : : : :- Project : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -27,23 +27,23 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- Project : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -53,24 +53,24 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -80,19 +80,19 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -102,20 +102,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -125,19 +125,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -147,7 +147,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/simplified.txt index 56bc32e144..dd6fd6a2ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - WholeStageCodegen (24) + WholeStageCodegen (18) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] @@ -11,17 +11,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] @@ -34,69 +34,65 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter @@ -106,25 +102,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter @@ -134,26 +130,26 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #14 - WholeStageCodegen (19) + WholeStageCodegen (14) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) + WholeStageCodegen (13) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #16 - WholeStageCodegen (16) + WholeStageCodegen (12) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -163,25 +159,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #17 - WholeStageCodegen (23) + WholeStageCodegen (17) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) + WholeStageCodegen (16) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #19 - WholeStageCodegen (20) + WholeStageCodegen (15) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/explain.txt index b2368c1fe0..a4f922dca8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/explain.txt @@ -612,7 +612,7 @@ Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (111) -+- * CometColumnarToRow (110) ++- CometNativeColumnarToRow (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -628,7 +628,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(110) CometColumnarToRow [codegen id : 1] +(110) CometNativeColumnarToRow Input [2]: [d_date_sk#22, d_year#23] (111) BroadcastExchange @@ -637,7 +637,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 BroadcastExchange (115) -+- * CometColumnarToRow (114) ++- CometNativeColumnarToRow (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -653,7 +653,7 @@ ReadSchema: struct Input [2]: [d_date_sk#49, d_year#50] Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2002)) AND isnotnull(d_date_sk#49)) -(114) CometColumnarToRow [codegen id : 1] +(114) CometNativeColumnarToRow Input [2]: [d_date_sk#49, d_year#50] (115) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt index 1be0816ee1..2336733a6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -47,7 +47,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/simplified.txt index 36be965b48..f984ee0702 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/simplified.txt @@ -27,11 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -51,11 +49,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt index b2368c1fe0..a4f922dca8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt @@ -612,7 +612,7 @@ Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (111) -+- * CometColumnarToRow (110) ++- CometNativeColumnarToRow (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -628,7 +628,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(110) CometColumnarToRow [codegen id : 1] +(110) CometNativeColumnarToRow Input [2]: [d_date_sk#22, d_year#23] (111) BroadcastExchange @@ -637,7 +637,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 BroadcastExchange (115) -+- * CometColumnarToRow (114) ++- CometNativeColumnarToRow (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -653,7 +653,7 @@ ReadSchema: struct Input [2]: [d_date_sk#49, d_year#50] Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2002)) AND isnotnull(d_date_sk#49)) -(114) CometColumnarToRow [codegen id : 1] +(114) CometNativeColumnarToRow Input [2]: [d_date_sk#49, d_year#50] (115) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/extended.txt index 1be0816ee1..2336733a6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -47,7 +47,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange 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 36be965b48..f984ee0702 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 @@ -27,11 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -51,11 +49,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/explain.txt index 5f766cebdc..00941aff02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (34) +CometNativeColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -188,14 +188,14 @@ Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -(34) CometColumnarToRow [codegen id : 2] +(34) CometNativeColumnarToRow Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometFilter (36) +- CometNativeScan parquet spark_catalog.default.date_dim (35) @@ -211,7 +211,7 @@ ReadSchema: struct Input [2]: [d_date_sk#18, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [2]: [d_date_sk#18, d_date#19] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt index d5dbd76871..f1d98d3059 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -18,7 +18,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/simplified.txt index 13760fcb0b..e28fe12099 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/simplified.txt @@ -1,45 +1,41 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - 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 [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] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #2 - WholeStageCodegen (1) - Filter [cs_warehouse_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] + 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 [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] + CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] + CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] + CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] + CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #2 + WholeStageCodegen (1) + Filter [cs_warehouse_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] + CometExchange [cr_order_number,cr_item_sk] #4 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_state] #5 + CometProject [w_state] [w_warehouse_sk,w_state] + CometFilter [w_warehouse_sk,w_state] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/explain.txt index ab95b81a9a..8e913d752e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -207,7 +207,7 @@ ReadSchema: struct Input [2]: [d_date_sk#18, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [2]: [d_date_sk#18, d_date#19] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt index 54a7b21486..8d2e593975 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/simplified.txt index 79e3556750..96423b87fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/simplified.txt @@ -19,11 +19,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] CometExchange [cr_order_number,cr_item_sk] #4 CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt index ab95b81a9a..8e913d752e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt @@ -191,7 +191,7 @@ Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -207,7 +207,7 @@ ReadSchema: struct Input [2]: [d_date_sk#18, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [2]: [d_date_sk#18, d_date#19] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/extended.txt index 54a7b21486..8d2e593975 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort 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 79e3556750..96423b87fb 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 @@ -19,11 +19,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] CometExchange [cr_order_number,cr_item_sk] #4 CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/explain.txt index 786423a455..abb88c99cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -102,6 +102,6 @@ Functions: [] Input [1]: [i_product_name#12] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [1]: [i_product_name#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/extended.txt index b40ebddb6f..6f7d077cf3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/simplified.txt index 2557f51684..76e150734c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometExchange [i_product_name] #1 CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + CometProject [i_product_name] [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometNativeScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] + CometExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/explain.txt index 71f0293c69..f591131ba7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -102,6 +102,6 @@ Functions: [] Input [1]: [i_product_name#12] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [1]: [i_product_name#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/extended.txt index 9d802b5033..a3c8dd9568 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/simplified.txt index 99c935aaa4..586ccd2852 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometExchange [i_product_name] #1 CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + CometProject [i_product_name] [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] + CometExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt index 71f0293c69..f591131ba7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -102,6 +102,6 @@ Functions: [] Input [1]: [i_product_name#12] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [1]: [i_product_name#12] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/extended.txt index 9d802b5033..a3c8dd9568 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange 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 99c935aaa4..586ccd2852 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometExchange [i_product_name] #1 CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + CometProject [i_product_name] [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] + CometExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/explain.txt index ec72e1b00f..11480e4796 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/simplified.txt index 96ab34454d..6ad3968250 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(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 [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] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(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 [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] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 + CometProject [i_category] [i_item_sk,i_category_id,i_category] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/explain.txt index 1a5c4758da..b87ec034c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_s Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/extended.txt index 5eaea983af..cebd2e28d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/simplified.txt index c7064b3ad8..d40373ef42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -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 [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 [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] - CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] +CometNativeColumnarToRow + CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + 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 [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] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [d_day_name] [d_date_sk,d_day_name] + CometFilter [d_date_sk,d_year,d_day_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/explain.txt index 835c6f13f7..3f27f98eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometTakeOrderedAndProject (46) +- CometProject (45) +- CometBroadcastHashJoin (44) @@ -13,11 +13,11 @@ : : : +- * Filter (16) : : : +- Window (15) : : : +- WindowGroupLimit (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometSort (12) : : : +- CometColumnarExchange (11) : : : +- WindowGroupLimit (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometSort (8) : : : +- CometFilter (7) : : : +- CometHashAggregate (6) @@ -32,11 +32,11 @@ : : +- * Filter (31) : : +- Window (30) : : +- WindowGroupLimit (29) - : : +- * CometColumnarToRow (28) + : : +- CometNativeColumnarToRow (28) : : +- CometSort (27) : : +- CometColumnarExchange (26) : : +- WindowGroupLimit (25) - : : +- * CometColumnarToRow (24) + : : +- CometNativeColumnarToRow (24) : : +- CometSort (23) : : +- CometFilter (22) : : +- CometHashAggregate (21) @@ -85,7 +85,7 @@ Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(9) CometColumnarToRow [codegen id : 1] +(9) CometNativeColumnarToRow Input [2]: [item_sk#7, rank_col#8] (10) WindowGroupLimit @@ -100,7 +100,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [2]: [item_sk#7, rank_col#8] (14) WindowGroupLimit @@ -111,11 +111,11 @@ Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final Input [2]: [item_sk#7, rank_col#8] Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] -(16) Filter [codegen id : 3] +(16) Filter [codegen id : 1] Input [3]: [item_sk#7, rank_col#8, rnk#11] Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(17) Project [codegen id : 3] +(17) Project [codegen id : 1] Output [2]: [item_sk#7, rnk#11] Input [3]: [item_sk#7, rank_col#8, rnk#11] @@ -143,7 +143,7 @@ Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0 Input [2]: [item_sk#16, rank_col#17] Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [2]: [item_sk#16, rank_col#17] (25) WindowGroupLimit @@ -158,7 +158,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [2]: [item_sk#16, rank_col#17] Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [2]: [item_sk#16, rank_col#17] (29) WindowGroupLimit @@ -169,11 +169,11 @@ Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final Input [2]: [item_sk#16, rank_col#17] Arguments: [rank(rank_col#17) windowspecdefinition(rank_col#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#18], [rank_col#17 DESC NULLS LAST] -(31) Filter [codegen id : 6] +(31) Filter [codegen id : 2] Input [3]: [item_sk#16, rank_col#17, rnk#18] Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [2]: [item_sk#16, rnk#18] Input [3]: [item_sk#16, rank_col#17, rnk#18] @@ -238,13 +238,13 @@ Arguments: [rnk#11, best_performing#24, worst_performing#25], [rnk#11, i_product Input [3]: [rnk#11, best_performing#24, worst_performing#25] Arguments: TakeOrderedAndProject(limit=100, orderBy=[rnk#11 ASC NULLS FIRST], output=[rnk#11,best_performing#24,worst_performing#25]), [rnk#11, best_performing#24, worst_performing#25], 100, 0, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#24, worst_performing#25] -(47) CometColumnarToRow [codegen id : 7] +(47) CometNativeColumnarToRow Input [3]: [rnk#11, best_performing#24, worst_performing#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (54) +CometNativeColumnarToRow (54) +- CometHashAggregate (53) +- CometExchange (52) +- CometHashAggregate (51) @@ -282,7 +282,7 @@ Input [3]: [ss_store_sk#27, sum#30, count#31] Keys [1]: [ss_store_sk#27] Functions [1]: [avg(UnscaledValue(ss_net_profit#28))] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [rank_col#32] Subquery:2 Hosting operator id = 22 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/extended.txt index 6595583fa5..c1b85fc86e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin @@ -12,15 +12,15 @@ CometColumnarToRow : : : +- Filter : : : +- Window : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometSort : : : +- CometColumnarExchange : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometSort : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -39,11 +39,11 @@ CometColumnarToRow : : +- Filter : : +- Window : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometFilter : : : +- ReusedSubquery diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/simplified.txt index 02d8749188..1d328a5bfd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/simplified.txt @@ -1,72 +1,60 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [rnk,best_performing,worst_performing] - CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] - CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] - CometProject [rnk,item_sk,i_product_name] - CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] - CometProject [item_sk,rnk,item_sk] - CometSortMergeJoin [item_sk,rnk,item_sk,rnk] - CometSort [item_sk,rnk] - CometColumnarExchange [rnk] #1 - WholeStageCodegen (3) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter +CometNativeColumnarToRow + CometTakeOrderedAndProject [rnk,best_performing,worst_performing] + CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] + CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] + CometProject [rnk,item_sk,i_product_name] + CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] + CometProject [item_sk,rnk,item_sk] + CometSortMergeJoin [item_sk,rnk,item_sk,rnk] + CometSort [item_sk,rnk] + CometColumnarExchange [rnk] #1 + WholeStageCodegen (1) + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WindowGroupLimit [rank_col] + CometNativeColumnarToRow + CometSort [item_sk,rank_col] + CometColumnarExchange #2 + WindowGroupLimit [rank_col] + CometNativeColumnarToRow CometSort [item_sk,rank_col] - CometColumnarExchange #2 - WindowGroupLimit [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #4 - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #3 - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometSort [item_sk,rnk] - CometColumnarExchange [rnk] #5 - WholeStageCodegen (6) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter + CometFilter [item_sk,rank_col] + Subquery #1 + CometNativeColumnarToRow + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_store_sk] #4 + 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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_item_sk] #3 + 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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometSort [item_sk,rnk] + CometColumnarExchange [rnk] #5 + WholeStageCodegen (2) + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WindowGroupLimit [rank_col] + CometNativeColumnarToRow + CometSort [item_sk,rank_col] + CometColumnarExchange #6 + WindowGroupLimit [rank_col] + CometNativeColumnarToRow CometSort [item_sk,rank_col] - CometColumnarExchange #6 - WindowGroupLimit [rank_col] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - ReusedSubquery [rank_col] #1 - 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] - CometFilter [i_item_sk,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_product_name] - ReusedExchange [i_item_sk,i_product_name] #7 + CometFilter [item_sk,rank_col] + ReusedSubquery [rank_col] #1 + 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] + CometFilter [i_item_sk,i_product_name] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_product_name] + ReusedExchange [i_item_sk,i_product_name] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/explain.txt index 3a1a49b654..950f3ff4e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * Project (34) @@ -19,22 +19,22 @@ TakeOrderedAndProject (39) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.web_sales (1) : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) + : : : : +- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.customer (4) : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometProject (12) : : : +- CometFilter (11) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) : : +- ReusedExchange (17) : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometProject (22) : +- CometFilter (21) : +- CometNativeScan parquet spark_catalog.default.item (20) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.item (27) @@ -48,10 +48,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 1] Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) @@ -66,20 +66,20 @@ ReadSchema: struct Input [2]: [c_customer_sk#7, c_current_addr_sk#8] Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [2]: [c_customer_sk#7, c_current_addr_sk#8] (7) BroadcastExchange Input [2]: [c_customer_sk#7, c_current_addr_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_bill_customer_sk#3] Right keys [1]: [c_customer_sk#7] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 1] Output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] @@ -98,33 +98,33 @@ Condition : isnotnull(ca_address_sk#9) Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#11, 10)) AS ca_zip#12] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] (14) BroadcastExchange Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#8] Right keys [1]: [ca_address_sk#9] Join type: Inner Join condition: None -(16) Project [codegen id : 6] +(16) Project [codegen id : 1] Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] (17) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#13] -(18) BroadcastHashJoin [codegen id : 6] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#5] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(19) Project [codegen id : 1] Output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] @@ -143,20 +143,20 @@ Condition : isnotnull(i_item_sk#14) Input [2]: [i_item_sk#14, i_item_id#15] Arguments: [i_item_sk#14, i_item_id#16], [i_item_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#15, 16)) AS i_item_id#16] -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [i_item_sk#14, i_item_id#16] (24) BroadcastExchange Input [2]: [i_item_sk#14, i_item_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#2] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(26) Project [codegen id : 1] Output [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16] Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#14, i_item_id#16] @@ -175,28 +175,28 @@ Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#17, i_item_id#18] Arguments: [i_item_id#19], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#18, 16)) AS i_item_id#19] -(30) CometColumnarToRow [codegen id : 5] +(30) CometNativeColumnarToRow Input [1]: [i_item_id#19] (31) BroadcastExchange Input [1]: [i_item_id#19] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 6] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [i_item_id#16] Right keys [1]: [i_item_id#19] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 6] +(33) Filter [codegen id : 1] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(34) Project [codegen id : 6] +(34) Project [codegen id : 1] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] -(35) HashAggregate [codegen id : 6] +(35) HashAggregate [codegen id : 1] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] @@ -207,10 +207,10 @@ Results [3]: [ca_zip#12, ca_city#10, sum#21] Input [3]: [ca_zip#12, ca_city#10, sum#21] Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 7] +(37) CometNativeColumnarToRow Input [3]: [ca_zip#12, ca_city#10, sum#21] -(38) HashAggregate [codegen id : 7] +(38) HashAggregate [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#21] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] @@ -225,7 +225,7 @@ Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometNativeScan parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2) Input [3]: [d_date_sk#13, d_year#24, d_qoy#25] Arguments: [d_date_sk#13], [d_date_sk#13] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt index c58a64e326..5b686ba177 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -19,31 +19,31 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/simplified.txt index 81ecde84b2..7a0bb62bd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) + WholeStageCodegen (1) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] @@ -23,42 +23,32 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + CometNativeColumnarToRow + CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_city,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/explain.txt index cf57f11d1e..062f9631df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/explain.txt @@ -34,7 +34,7 @@ TakeOrderedAndProject (39) : +- CometFilter (21) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) @@ -160,7 +160,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] -(26) CometColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 1] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -178,28 +178,28 @@ Condition : i_item_sk#19 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#19, i_item_id#20] Arguments: [i_item_id#21], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#20, 16)) AS i_item_id#21] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [i_item_id#21] (31) BroadcastExchange Input [1]: [i_item_id#21] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] -(32) BroadcastHashJoin [codegen id : 2] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [i_item_id#18] Right keys [1]: [i_item_id#21] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 2] +(33) Filter [codegen id : 1] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(34) Project [codegen id : 2] +(34) Project [codegen id : 1] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -(35) HashAggregate [codegen id : 2] +(35) HashAggregate [codegen id : 1] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] @@ -210,10 +210,10 @@ Results [3]: [ca_zip#12, ca_city#10, sum#23] Input [3]: [ca_zip#12, ca_city#10, sum#23] Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(37) CometColumnarToRow [codegen id : 3] +(37) CometColumnarToRow [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#23] -(38) HashAggregate [codegen id : 3] +(38) HashAggregate [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#23] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] @@ -228,7 +228,7 @@ Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -249,7 +249,7 @@ Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2) Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt index 4a57091056..23662b8aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/simplified.txt index d78fc716ab..2d2ba0c558 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] @@ -23,12 +23,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] @@ -46,9 +44,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [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] + CometNativeColumnarToRow + CometProject [i_item_id] [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt index cf57f11d1e..062f9631df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt @@ -34,7 +34,7 @@ TakeOrderedAndProject (39) : +- CometFilter (21) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) @@ -160,7 +160,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] -(26) CometColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 1] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -178,28 +178,28 @@ Condition : i_item_sk#19 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#19, i_item_id#20] Arguments: [i_item_id#21], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#20, 16)) AS i_item_id#21] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [i_item_id#21] (31) BroadcastExchange Input [1]: [i_item_id#21] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] -(32) BroadcastHashJoin [codegen id : 2] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [i_item_id#18] Right keys [1]: [i_item_id#21] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 2] +(33) Filter [codegen id : 1] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(34) Project [codegen id : 2] +(34) Project [codegen id : 1] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -(35) HashAggregate [codegen id : 2] +(35) HashAggregate [codegen id : 1] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] @@ -210,10 +210,10 @@ Results [3]: [ca_zip#12, ca_city#10, sum#23] Input [3]: [ca_zip#12, ca_city#10, sum#23] Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(37) CometColumnarToRow [codegen id : 3] +(37) CometColumnarToRow [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#23] -(38) HashAggregate [codegen id : 3] +(38) HashAggregate [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#23] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] @@ -228,7 +228,7 @@ Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -249,7 +249,7 @@ Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2) Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/extended.txt index 4a57091056..23662b8aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt index d78fc716ab..2d2ba0c558 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] @@ -23,12 +23,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] @@ -46,9 +44,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [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] + CometNativeColumnarToRow + CometProject [i_item_id] [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/explain.txt index 4630317250..e344ef2e6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (41) :- * Project (37) : +- * BroadcastHashJoin Inner BuildRight (36) : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) + : : +- CometNativeColumnarToRow (29) : : +- CometColumnarExchange (28) : : +- * HashAggregate (27) : : +- * Project (26) @@ -21,21 +21,21 @@ TakeOrderedAndProject (41) : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : +- ReusedExchange (4) : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) + : : : : +- CometNativeColumnarToRow (10) : : : : +- CometProject (9) : : : : +- CometFilter (8) : : : : +- CometNativeScan parquet spark_catalog.default.store (7) : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) + : : : +- CometNativeColumnarToRow (17) : : : +- CometProject (16) : : : +- CometFilter (15) : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometFilter (22) : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) + : +- CometNativeColumnarToRow (34) : +- CometProject (33) : +- CometFilter (32) : +- CometNativeScan parquet spark_catalog.default.customer (31) @@ -50,23 +50,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#10] -(5) BroadcastHashJoin [codegen id : 5] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(6) Project [codegen id : 5] +(6) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] @@ -85,20 +85,20 @@ Condition : (s_city#12 IN (Fairview,Midway) AND isnotnull(s_store_sk#11)) Input [2]: [s_store_sk#11, s_city#12] Arguments: [s_store_sk#11], [s_store_sk#11] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#11] (11) BroadcastExchange Input [1]: [s_store_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11] @@ -117,20 +117,20 @@ Condition : (((hd_dep_count#14 = 4) OR (hd_vehicle_count#15 = 3)) AND isnotnull( Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] Arguments: [hd_demo_sk#13], [hd_demo_sk#13] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#13] (18) BroadcastExchange Input [1]: [hd_demo_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#13] @@ -145,24 +145,24 @@ ReadSchema: struct Input [2]: [ca_address_sk#16, ca_city#17] Condition : (isnotnull(ca_address_sk#16) AND isnotnull(ca_city#17)) -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [ca_address_sk#16, ca_city#17] (24) BroadcastExchange Input [2]: [ca_address_sk#16, ca_city#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(26) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#16, ca_city#17] -(27) HashAggregate [codegen id : 5] +(27) HashAggregate [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] @@ -173,10 +173,10 @@ Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, su Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometColumnarToRow [codegen id : 8] +(29) CometNativeColumnarToRow Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -(30) HashAggregate [codegen id : 8] +(30) HashAggregate [codegen id : 2] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] @@ -198,33 +198,33 @@ Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] Arguments: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32], [c_customer_sk#27, c_current_addr_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#29, 20)) AS c_first_name#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#30, 30)) AS c_last_name#32] -(34) CometColumnarToRow [codegen id : 6] +(34) CometNativeColumnarToRow Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] (35) BroadcastExchange Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#27] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 2] Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32] Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] (38) ReusedExchange [Reuses operator id: 24] Output [2]: [ca_address_sk#33, ca_city#34] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_current_addr_sk#28] Right keys [1]: [ca_address_sk#33] Join type: Inner Join condition: NOT (ca_city#34 = bought_city#24) -(40) Project [codegen id : 8] +(40) Project [codegen id : 2] Output [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] @@ -236,7 +236,7 @@ Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometNativeScan parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((d_dow#36 IN (6,0) AND d_year#35 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#35, d_dow#36] Arguments: [d_date_sk#10], [d_date_sk#10] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt index c122bf3803..cebcd85144 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -20,36 +20,36 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/simplified.txt index b4b78689b1..2b3e7984ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -24,44 +24,34 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometNativeColumnarToRow + CometFilter [ca_address_sk,ca_city] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt index 08bba41c02..466c9c615f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt @@ -225,7 +225,7 @@ Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt index e646cb620a..59123d0f6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt index 08bba41c02..466c9c615f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt @@ -225,7 +225,7 @@ Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e646cb620a..59123d0f6f 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/explain.txt index 5956a3b4a6..3647819adf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.store_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.store (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCode Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.store_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#6] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] (17) BroadcastExchange Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#7] Right keys [1]: [s_store_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] @@ -151,10 +151,10 @@ Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_yea Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] (27) Window Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] (32) ReusedExchange [Reuses operator id: 21] Output [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] Keys [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29] Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#24, i_brand#25, s_store_name#26, s_compan Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] Arguments: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32], [i_category#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, s_store_name#26 ASC NULLS FIRST, s_company_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] (38) Window Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#24, i_brand#25, s_store_name#26, s_company_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] Input [8]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32, rn#33] @@ -228,13 +228,13 @@ Input [8]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_yea Input [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, (rn#33 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#32] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] @@ -245,14 +245,14 @@ Output [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_ye Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] Arguments: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40], [i_category#34 ASC NULLS FIRST, i_brand#35 ASC NULLS FIRST, s_store_name#36 ASC NULLS FIRST, s_company_name#37 ASC NULLS FIRST, d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] (46) Window Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] Arguments: [rank(d_year#38, d_moy#39) windowspecdefinition(i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#34, i_brand#35, s_store_name#36, s_company_name#37], [d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [6]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] Input [8]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40, rn#41] @@ -260,13 +260,13 @@ Input [8]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_yea Input [6]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, (rn#41 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#32 AS psum#42, sum_sales#40 AS nsum#43] Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#32, i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt index 092083e465..ab44bbef5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/simplified.txt index 017805e5b2..139a34baee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk,s_store_name,s_company_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt index 5b6a94ab57..1a6bcc1d18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt index 19124a9e20..56fdda3162 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt index 5b6a94ab57..1a6bcc1d18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 19124a9e20..56fdda3162 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/explain.txt index 3f674fa5c8..e5038ab586 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * HashAggregate (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometColumnarExchange (28) +- * HashAggregate (27) +- * Project (26) @@ -15,16 +15,16 @@ : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) + : : : +- CometNativeColumnarToRow (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.store (4) : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) + : : +- CometNativeColumnarToRow (13) : : +- CometProject (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (10) : +- BroadcastExchange (21) - : +- * CometColumnarToRow (20) + : +- CometNativeColumnarToRow (20) : +- CometProject (19) : +- CometFilter (18) : +- CometNativeScan parquet spark_catalog.default.customer_address (17) @@ -39,10 +39,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) @@ -57,20 +57,20 @@ ReadSchema: struct Input [1]: [s_store_sk#9] Condition : isnotnull(s_store_sk#9) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [s_store_sk#9] (7) BroadcastExchange Input [1]: [s_store_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 5] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#9] Join type: Inner Join condition: None -(9) Project [codegen id : 5] +(9) Project [codegen id : 1] Output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] @@ -89,20 +89,20 @@ Condition : (isnotnull(cd_demo_sk#10) AND ((((static_invoke(CharVarcharCodegenUt Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) AS cd_marital_status#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) AS cd_education_status#14] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] (14) BroadcastExchange Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 5] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) -(16) Project [codegen id : 5] +(16) Project [codegen id : 1] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] @@ -121,37 +121,37 @@ Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) AS ca_state#18] -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [2]: [ca_address_sk#15, ca_state#18] (21) BroadcastExchange Input [2]: [ca_address_sk#15, ca_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 5] +(22) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#15] Join type: Inner Join condition: ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))) -(23) Project [codegen id : 5] +(23) Project [codegen id : 1] Output [2]: [ss_quantity#4, ss_sold_date_sk#7] Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] (24) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#19] -(25) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(26) Project [codegen id : 1] Output [1]: [ss_quantity#4] Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] -(27) HashAggregate [codegen id : 5] +(27) HashAggregate [codegen id : 1] Input [1]: [ss_quantity#4] Keys: [] Functions [1]: [partial_sum(ss_quantity#4)] @@ -162,10 +162,10 @@ Results [1]: [sum#21] Input [1]: [sum#21] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometColumnarToRow [codegen id : 6] +(29) CometNativeColumnarToRow Input [1]: [sum#21] -(30) HashAggregate [codegen id : 6] +(30) HashAggregate [codegen id : 2] Input [1]: [sum#21] Keys: [] Functions [1]: [sum(ss_quantity#4)] @@ -176,7 +176,7 @@ Results [1]: [sum(ss_quantity#4)#22 AS sum(ss_quantity)#23] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#24) AND (d_year#24 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#19, d_year#24] Arguments: [d_date_sk#19], [d_date_sk#19] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt index 1ef8426231..5e6a26863a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,26 +15,26 @@ HashAggregate : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/simplified.txt index 0cd7e900b0..b258895829 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (6) +WholeStageCodegen (2) HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_quantity] [sum,sum] Project [ss_quantity] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -19,34 +19,26 @@ WholeStageCodegen (6) Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/explain.txt index e481658923..43a8d667da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/explain.txt @@ -171,7 +171,7 @@ Input [1]: [sum(ss_quantity)#22] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -192,7 +192,7 @@ Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#19, d_year#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt index 57fbc2d035..347702b724 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/simplified.txt index e29e101c7e..52cdd88b76 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/simplified.txt @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt index e481658923..43a8d667da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt @@ -171,7 +171,7 @@ Input [1]: [sum(ss_quantity)#22] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -192,7 +192,7 @@ Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#19, d_year#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/extended.txt index 57fbc2d035..347702b724 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e29e101c7e..52cdd88b76 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 @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/explain.txt index 0a43ebc390..90cb3f712c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (84) +CometNativeColumnarToRow (84) +- CometTakeOrderedAndProject (83) +- CometHashAggregate (82) +- CometColumnarExchange (81) @@ -10,11 +10,11 @@ : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometSort (20) : +- CometColumnarExchange (19) : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometColumnarExchange (16) : +- * HashAggregate (15) : +- * Project (14) @@ -26,7 +26,7 @@ : : : +- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) + : : +- CometNativeColumnarToRow (9) : : +- CometProject (8) : : +- CometFilter (7) : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) @@ -36,11 +36,11 @@ : +- Window (50) : +- * Sort (49) : +- Window (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometSort (46) : +- CometColumnarExchange (45) : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -52,7 +52,7 @@ : : : +- * Filter (29) : : : +- * ColumnarToRow (28) : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) + : : +- CometNativeColumnarToRow (35) : : +- CometProject (34) : : +- CometFilter (33) : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) @@ -62,11 +62,11 @@ +- Window (76) +- * Sort (75) +- Window (74) - +- * CometColumnarToRow (73) + +- CometNativeColumnarToRow (73) +- CometSort (72) +- CometColumnarExchange (71) +- * HashAggregate (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometColumnarExchange (68) +- * HashAggregate (67) +- * Project (66) @@ -78,7 +78,7 @@ : : +- * Filter (55) : : +- * ColumnarToRow (54) : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) + : +- CometNativeColumnarToRow (61) : +- CometProject (60) : +- CometFilter (59) : +- CometNativeScan parquet spark_catalog.default.store_returns (58) @@ -123,33 +123,33 @@ Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AN Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometColumnarToRow +(9) CometNativeColumnarToRow Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [2]: [ws_order_number#2, ws_item_sk#1] Right keys [2]: [wr_order_number#9, wr_item_sk#8] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] (12) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#13] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#6] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 2] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -160,10 +160,10 @@ Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty# Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -178,14 +178,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 5] +(21) CometNativeColumnarToRow Input [3]: [item#30, return_ratio#31, currency_ratio#32] (22) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 4] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 @@ -193,11 +193,11 @@ Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 5] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(26) Project [codegen id : 7] +(26) Project [codegen id : 5] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -209,14 +209,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -(29) Filter [codegen id : 8] +(29) Filter [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(30) Project [codegen id : 8] +(30) Project [codegen id : 6] Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] @@ -239,33 +239,33 @@ Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.0 Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(35) CometColumnarToRow +(35) CometNativeColumnarToRow Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 7] Left keys [2]: [cs_order_number#37, cs_item_sk#36] Right keys [2]: [cr_order_number#43, cr_item_sk#42] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 7] Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] (38) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#47] -(39) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#47] Join type: Inner Join condition: None -(40) Project [codegen id : 10] +(40) Project [codegen id : 7] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] -(41) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -276,10 +276,10 @@ Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometColumnarToRow [codegen id : 11] +(43) CometNativeColumnarToRow Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 8] Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -294,14 +294,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 12] +(47) CometNativeColumnarToRow Input [3]: [item#64, return_ratio#65, currency_ratio#66] (48) Window Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] -(49) Sort [codegen id : 13] +(49) Sort [codegen id : 9] Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 @@ -309,11 +309,11 @@ Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] -(51) Filter [codegen id : 14] +(51) Filter [codegen id : 10] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) -(52) Project [codegen id : 14] +(52) Project [codegen id : 10] Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] @@ -325,14 +325,14 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 15] +(54) ColumnarToRow [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -(55) Filter [codegen id : 15] +(55) Filter [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) -(56) Project [codegen id : 15] +(56) Project [codegen id : 11] Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] @@ -355,33 +355,33 @@ Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AN Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(61) CometColumnarToRow +(61) CometNativeColumnarToRow Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(62) BroadcastHashJoin [codegen id : 17] +(62) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(63) Project [codegen id : 12] Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] (64) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#81] -(65) BroadcastHashJoin [codegen id : 17] +(65) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ss_sold_date_sk#75] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(66) Project [codegen id : 12] Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] -(67) HashAggregate [codegen id : 17] +(67) HashAggregate [codegen id : 12] Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Keys [1]: [ss_item_sk#70] Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -392,10 +392,10 @@ Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(69) CometColumnarToRow [codegen id : 18] +(69) CometNativeColumnarToRow Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -(70) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 13] Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Keys [1]: [ss_item_sk#70] Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -410,14 +410,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [3]: [item#98, return_ratio#99, currency_ratio#100] (74) Window Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] -(75) Sort [codegen id : 20] +(75) Sort [codegen id : 14] Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 @@ -425,17 +425,17 @@ Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] -(77) Filter [codegen id : 21] +(77) Filter [codegen id : 15] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) -(78) Project [codegen id : 21] +(78) Project [codegen id : 15] Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] (79) Union -(80) HashAggregate [codegen id : 22] +(80) HashAggregate [codegen id : 16] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] @@ -455,14 +455,14 @@ Functions: [] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) CometColumnarToRow [codegen id : 23] +(84) CometNativeColumnarToRow Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometNativeScan parquet spark_catalog.default.date_dim (85) @@ -483,7 +483,7 @@ Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 Input [3]: [d_date_sk#13, d_year#104, d_moy#105] Arguments: [d_date_sk#13], [d_date_sk#13] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt index 6c4ded1515..0090739d55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -9,11 +9,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -27,16 +27,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -45,11 +45,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -62,12 +62,12 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -76,11 +76,11 @@ CometColumnarToRow +- Window +- Sort +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -93,12 +93,12 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/simplified.txt index 047269434d..018aec1d47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/simplified.txt @@ -1,140 +1,130 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + CometNativeColumnarToRow + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #6 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [cs_item_sk] #7 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (15) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #9 + WholeStageCodegen (13) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (12) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (11) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt index 338f5f4201..0ecba1505f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt index ca80833ee5..3353e1c331 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt index 338f5f4201..0ecba1505f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ca80833ee5..3353e1c331 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 @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/explain.txt index 39114bd417..57e5e79a4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == TakeOrderedAndProject (81) +- * HashAggregate (80) - +- * CometColumnarToRow (79) + +- CometNativeColumnarToRow (79) +- CometColumnarExchange (78) +- * HashAggregate (77) +- * Expand (76) +- Union (75) :- * HashAggregate (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometColumnarExchange (21) : +- * HashAggregate (20) : +- * Project (19) @@ -25,12 +25,12 @@ TakeOrderedAndProject (81) : : : +- Scan parquet spark_catalog.default.store_returns (5) : : +- ReusedExchange (10) : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometProject (15) : +- CometFilter (14) : +- CometNativeScan parquet spark_catalog.default.store (13) :- * HashAggregate (46) - : +- * CometColumnarToRow (45) + : +- CometNativeColumnarToRow (45) : +- CometColumnarExchange (44) : +- * HashAggregate (43) : +- * Project (42) @@ -48,12 +48,12 @@ TakeOrderedAndProject (81) : : : +- Scan parquet spark_catalog.default.catalog_returns (28) : : +- ReusedExchange (33) : +- BroadcastExchange (40) - : +- * CometColumnarToRow (39) + : +- CometNativeColumnarToRow (39) : +- CometProject (38) : +- CometFilter (37) : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) +- * HashAggregate (74) - +- * CometColumnarToRow (73) + +- CometNativeColumnarToRow (73) +- CometColumnarExchange (72) +- * HashAggregate (71) +- * Project (70) @@ -70,13 +70,13 @@ TakeOrderedAndProject (81) : : :- BroadcastExchange (53) : : : +- * ColumnarToRow (52) : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : +- * CometColumnarToRow (57) + : : +- CometNativeColumnarToRow (57) : : +- CometProject (56) : : +- CometFilter (55) : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) : +- ReusedExchange (61) +- BroadcastExchange (68) - +- * CometColumnarToRow (67) + +- CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometNativeScan parquet spark_catalog.default.web_site (64) @@ -125,13 +125,13 @@ Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_s (10) ReusedExchange [Reuses operator id: 86] Output [1]: [d_date_sk#22] -(11) BroadcastHashJoin [codegen id : 5] +(11) BroadcastHashJoin [codegen id : 3] Left keys [1]: [date_sk#7] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(12) Project [codegen id : 5] +(12) Project [codegen id : 3] Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] @@ -150,24 +150,24 @@ Condition : isnotnull(s_store_sk#23) Input [2]: [s_store_sk#23, s_store_id#24] Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#24, 16)) AS s_store_id#25] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#23, s_store_id#25] (17) BroadcastExchange Input [2]: [s_store_sk#23, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 5] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [store_sk#6] Right keys [1]: [s_store_sk#23] Join type: Inner Join condition: None -(19) Project [codegen id : 5] +(19) Project [codegen id : 3] Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] -(20) HashAggregate [codegen id : 5] +(20) HashAggregate [codegen id : 3] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] @@ -178,10 +178,10 @@ Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(23) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 4] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -196,14 +196,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] +(25) ColumnarToRow [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -(26) Filter [codegen id : 7] +(26) Filter [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Condition : isnotnull(cs_catalog_page_sk#43) -(27) Project [codegen id : 7] +(27) Project [codegen id : 5] Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] @@ -215,14 +215,14 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 8] +(29) ColumnarToRow [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -(30) Filter [codegen id : 8] +(30) Filter [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(31) Project [codegen id : 8] +(31) Project [codegen id : 6] Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] @@ -231,13 +231,13 @@ Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_retur (33) ReusedExchange [Reuses operator id: 86] Output [1]: [d_date_sk#63] -(34) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(35) Project [codegen id : 11] +(35) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -256,24 +256,24 @@ Condition : isnotnull(cp_catalog_page_sk#64) Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#65, 16)) AS cp_catalog_page_id#66] -(39) CometColumnarToRow [codegen id : 10] +(39) CometNativeColumnarToRow Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] (40) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(41) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(42) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] @@ -284,10 +284,10 @@ Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(45) CometColumnarToRow [codegen id : 12] +(45) CometNativeColumnarToRow Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -(46) HashAggregate [codegen id : 12] +(46) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -302,14 +302,14 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 13] +(48) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(49) Filter [codegen id : 13] +(49) Filter [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Condition : isnotnull(ws_web_site_sk#84) -(50) Project [codegen id : 13] +(50) Project [codegen id : 9] Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] @@ -320,7 +320,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 14] +(52) ColumnarToRow [codegen id : 10] Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (53) BroadcastExchange @@ -342,16 +342,16 @@ Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND i Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(57) CometColumnarToRow +(57) CometNativeColumnarToRow Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(58) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 11] Left keys [2]: [wr_item_sk#94, wr_order_number#95] Right keys [2]: [ws_item_sk#99, ws_order_number#101] Join type: Inner Join condition: None -(59) Project [codegen id : 15] +(59) Project [codegen id : 11] Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] @@ -360,13 +360,13 @@ Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, (61) ReusedExchange [Reuses operator id: 86] Output [1]: [d_date_sk#109] -(62) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 12] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(63) Project [codegen id : 18] +(63) Project [codegen id : 12] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -385,24 +385,24 @@ Condition : isnotnull(web_site_sk#110) Input [2]: [web_site_sk#110, web_site_id#111] Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#111, 16)) AS web_site_id#112] -(67) CometColumnarToRow [codegen id : 17] +(67) CometNativeColumnarToRow Input [2]: [web_site_sk#110, web_site_id#112] (68) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(69) BroadcastHashJoin [codegen id : 18] +(69) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(70) Project [codegen id : 18] +(70) Project [codegen id : 12] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] -(71) HashAggregate [codegen id : 18] +(71) HashAggregate [codegen id : 12] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Keys [1]: [web_site_id#112] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] @@ -413,10 +413,10 @@ Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -(74) HashAggregate [codegen id : 19] +(74) HashAggregate [codegen id : 13] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Keys [1]: [web_site_id#112] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] @@ -425,11 +425,11 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales# (75) Union -(76) Expand [codegen id : 20] +(76) Expand [codegen id : 14] Input [5]: [sales#38, returns#39, profit#40, channel#41, id#42] Arguments: [[sales#38, returns#39, profit#40, channel#41, id#42, 0], [sales#38, returns#39, profit#40, channel#41, null, 1], [sales#38, returns#39, profit#40, null, null, 3]], [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] -(77) HashAggregate [codegen id : 20] +(77) HashAggregate [codegen id : 14] Input [6]: [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] Keys [3]: [channel#130, id#131, spark_grouping_id#132] Functions [3]: [partial_sum(sales#38), partial_sum(returns#39), partial_sum(profit#40)] @@ -440,10 +440,10 @@ Results [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 21] +(79) CometNativeColumnarToRow Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -(80) HashAggregate [codegen id : 21] +(80) HashAggregate [codegen id : 15] Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Keys [3]: [channel#130, id#131, spark_grouping_id#132] Functions [3]: [sum(sales#38), sum(returns#39), sum(profit#40)] @@ -458,7 +458,7 @@ Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel# Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (86) -+- * CometColumnarToRow (85) ++- CometNativeColumnarToRow (85) +- CometProject (84) +- CometFilter (83) +- CometNativeScan parquet spark_catalog.default.date_dim (82) @@ -479,7 +479,7 @@ Condition : (((isnotnull(d_date#151) AND (d_date#151 >= 2000-08-23)) AND (d_date Input [2]: [d_date_sk#22, d_date#151] Arguments: [d_date_sk#22], [d_date_sk#22] -(85) CometColumnarToRow [codegen id : 1] +(85) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (86) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt index 60352ef90a..15ad014521 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -30,17 +30,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -59,17 +59,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.catalog_page +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -88,17 +88,17 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/simplified.txt index 9dc4fc19c7..d84ba471ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (21) + WholeStageCodegen (15) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (20) + WholeStageCodegen (14) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_id] #2 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] BroadcastHashJoin [store_sk,s_store_sk] @@ -30,12 +30,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] WholeStageCodegen (2) Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_store_sk] @@ -47,18 +45,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cp_catalog_page_id] #5 - WholeStageCodegen (11) + WholeStageCodegen (7) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] BroadcastHashJoin [page_sk,cp_catalog_page_sk] @@ -66,14 +62,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (5) Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] Filter [cs_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) + WholeStageCodegen (6) Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] Filter [cr_catalog_page_sk] ColumnarToRow @@ -84,18 +80,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) + CometNativeColumnarToRow + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (13) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [web_site_id] #7 - WholeStageCodegen (18) + WholeStageCodegen (12) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] BroadcastHashJoin [wsr_web_site_sk,web_site_sk] @@ -103,25 +97,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union - WholeStageCodegen (13) + WholeStageCodegen (9) Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (15) + WholeStageCodegen (11) Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] InputAdapter BroadcastExchange #8 - WholeStageCodegen (14) + WholeStageCodegen (10) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] @@ -129,9 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometNativeColumnarToRow + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt index 6f23a02597..0418af61c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt @@ -411,7 +411,7 @@ Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -432,7 +432,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt index d128acaf06..76c166e7de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt index 6fff89a25c..aa3a803ef1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt index 6f23a02597..0418af61c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt @@ -411,7 +411,7 @@ Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -432,7 +432,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt index d128acaf06..76c166e7de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6fff89a25c..aa3a803ef1 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/explain.txt index 4001dbe16b..ed2d8b619d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (30) +- * HashAggregate (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometColumnarExchange (27) +- * HashAggregate (26) +- * Project (25) @@ -12,7 +12,7 @@ TakeOrderedAndProject (30) : : +- * BroadcastHashJoin Inner BuildRight (15) : : :- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * CometColumnarToRow (3) + : : : :- CometNativeColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (7) @@ -20,12 +20,12 @@ TakeOrderedAndProject (30) : : : +- * ColumnarToRow (5) : : : +- Scan parquet spark_catalog.default.store_returns (4) : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) + : : +- CometNativeColumnarToRow (13) : : +- CometProject (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.store (10) : +- BroadcastExchange (20) - : +- * CometColumnarToRow (19) + : +- CometNativeColumnarToRow (19) : +- CometFilter (18) : +- CometNativeScan parquet spark_catalog.default.date_dim (17) +- ReusedExchange (23) @@ -43,7 +43,7 @@ ReadSchema: struct Input [1]: [d_date_sk#26] Condition : isnotnull(d_date_sk#26) -(19) CometColumnarToRow [codegen id : 3] +(19) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (20) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(21) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(22) Project [codegen id : 5] +(22) Project [codegen id : 2] Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] (23) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#27] -(24) BroadcastHashJoin [codegen id : 5] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#9] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(25) Project [codegen id : 5] +(25) Project [codegen id : 2] Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 2] Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] @@ -159,10 +159,10 @@ Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_na Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] -(29) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 3] Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] @@ -177,7 +177,7 @@ Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRS Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -198,7 +198,7 @@ Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2 Input [3]: [d_date_sk#27, d_year#48, d_moy#49] Arguments: [d_date_sk#27], [d_date_sk#27] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#27] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt index 6ece21441c..ab41730455 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- BroadcastExchange @@ -20,21 +20,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/simplified.txt index 186c041b09..fddccaf3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -14,8 +14,8 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] InputAdapter @@ -27,26 +27,20 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometNativeColumnarToRow + CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk] + CometNativeColumnarToRow + CometFilter [d_date_sk] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk] InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/explain.txt index d55e867987..1c019ec8f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/explain.txt @@ -172,7 +172,7 @@ Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -193,7 +193,7 @@ Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2 Input [3]: [d_date_sk#27, d_year#28, d_moy#29] Arguments: [d_date_sk#27], [d_date_sk#27] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#27] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt index 0fbba7d1e8..958b844c9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/simplified.txt index 98a44eaf40..fd3cd6eaa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt index d55e867987..1c019ec8f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt @@ -172,7 +172,7 @@ Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -193,7 +193,7 @@ Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2 Input [3]: [d_date_sk#27, d_year#28, d_moy#29] Arguments: [d_date_sk#27], [d_date_sk#27] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#27] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/extended.txt index 0fbba7d1e8..958b844c9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 98a44eaf40..fd3cd6eaa5 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/explain.txt index 9af40029c8..508b20f435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (42) +- * Filter (41) +- Window (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometSort (38) +- CometExchange (37) +- CometProject (36) @@ -11,11 +11,11 @@ TakeOrderedAndProject (42) : +- CometColumnarExchange (16) : +- * Project (15) : +- Window (14) - : +- * CometColumnarToRow (13) + : +- CometNativeColumnarToRow (13) : +- CometSort (12) : +- CometColumnarExchange (11) : +- * HashAggregate (10) - : +- * CometColumnarToRow (9) + : +- CometNativeColumnarToRow (9) : +- CometColumnarExchange (8) : +- * HashAggregate (7) : +- * Project (6) @@ -28,11 +28,11 @@ TakeOrderedAndProject (42) +- CometColumnarExchange (33) +- * Project (32) +- Window (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometSort (29) +- CometColumnarExchange (28) +- * HashAggregate (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometColumnarExchange (25) +- * HashAggregate (24) +- * Project (23) @@ -51,27 +51,27 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 47] Output [2]: [d_date_sk#5, d_date#6] -(5) BroadcastHashJoin [codegen id : 2] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 2] +(6) Project [codegen id : 1] Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -(7) HashAggregate [codegen id : 2] +(7) HashAggregate [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] @@ -82,10 +82,10 @@ Results [3]: [ws_item_sk#1, d_date#6, sum#8] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 3] +(9) CometNativeColumnarToRow Input [3]: [ws_item_sk#1, d_date#6, sum#8] -(10) HashAggregate [codegen id : 3] +(10) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] @@ -100,14 +100,14 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 4] +(13) CometNativeColumnarToRow Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] (14) Window Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 5] +(15) Project [codegen id : 3] Output [3]: [item_sk#10, d_date#6, cume_sales#12] Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] @@ -127,27 +127,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 4] Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -(20) Filter [codegen id : 7] +(20) Filter [codegen id : 4] Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) (21) ReusedExchange [Reuses operator id: 47] Output [2]: [d_date_sk#16, d_date#17] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#15] Right keys [1]: [d_date_sk#16] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 4] Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16, d_date#17] -(24) HashAggregate [codegen id : 7] +(24) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] Keys [2]: [ss_item_sk#13, d_date#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] @@ -158,10 +158,10 @@ Results [3]: [ss_item_sk#13, d_date#17, sum#19] Input [3]: [ss_item_sk#13, d_date#17, sum#19] Arguments: hashpartitioning(ss_item_sk#13, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometColumnarToRow [codegen id : 8] +(26) CometNativeColumnarToRow Input [3]: [ss_item_sk#13, d_date#17, sum#19] -(27) HashAggregate [codegen id : 8] +(27) HashAggregate [codegen id : 5] Input [3]: [ss_item_sk#13, d_date#17, sum#19] Keys [2]: [ss_item_sk#13, d_date#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] @@ -176,14 +176,14 @@ Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumna Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] Arguments: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(30) CometColumnarToRow [codegen id : 9] +(30) CometNativeColumnarToRow Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] (31) Window Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#17 ASC NULLS FIRST] -(32) Project [codegen id : 10] +(32) Project [codegen id : 6] Output [3]: [item_sk#21, d_date#17, cume_sales#23] Input [5]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13, cume_sales#23] @@ -212,14 +212,14 @@ Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] -(39) CometColumnarToRow [codegen id : 11] +(39) CometNativeColumnarToRow Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] (40) Window Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(41) Filter [codegen id : 12] +(41) Filter [codegen id : 7] Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) @@ -231,7 +231,7 @@ Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (47) -+- * CometColumnarToRow (46) ++- CometNativeColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometNativeScan parquet spark_catalog.default.date_dim (43) @@ -252,7 +252,7 @@ Condition : (((isnotnull(d_month_seq#30) AND (d_month_seq#30 >= 1200)) AND (d_mo Input [3]: [d_date_sk#5, d_date#6, d_month_seq#30] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(46) CometColumnarToRow [codegen id : 1] +(46) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt index 9dc007f5e9..db4e38a1bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -10,11 +10,11 @@ TakeOrderedAndProject : +- CometColumnarExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -24,12 +24,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,11 +37,11 @@ TakeOrderedAndProject +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -51,7 +51,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/simplified.txt index 51d3f11ae6..1029c768d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/simplified.txt @@ -1,73 +1,65 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (12) + WholeStageCodegen (7) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #6 - WholeStageCodegen (10) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #8 - WholeStageCodegen (7) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + CometNativeColumnarToRow + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (3) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk,d_date] #4 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #6 + WholeStageCodegen (6) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #7 + WholeStageCodegen (5) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk,d_date] #8 + WholeStageCodegen (4) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt index 683cc25d24..716f297aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt @@ -224,7 +224,7 @@ Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -245,7 +245,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt index 66c5717cc7..8a2badfaff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt index a5af2e114b..ba02a3e822 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt @@ -30,12 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt index 683cc25d24..716f297aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt @@ -224,7 +224,7 @@ Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -245,7 +245,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt index 66c5717cc7..8a2badfaff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a5af2e114b..ba02a3e822 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 @@ -30,12 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/explain.txt index 9de09937ac..5f58cc0a3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/simplified.txt index cee223d027..f5de3784b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_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 [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] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [d_year,brand_id,brand,ext_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 [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] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/explain.txt index 8f5f268b10..888f78aa93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (30) +- * Project (29) +- * Filter (28) +- Window (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -16,7 +16,7 @@ TakeOrderedAndProject (30) : +- * BroadcastHashJoin Inner BuildRight (12) : :- * Project (10) : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject (30) : : +- Scan parquet spark_catalog.default.store_sales (5) : +- ReusedExchange (11) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -46,7 +46,7 @@ Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [2]: [i_item_sk#1, i_manufact_id#5] (5) Scan parquet spark_catalog.default.store_sales @@ -68,26 +68,26 @@ Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] (11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#15, d_qoy#16] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#13] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16] Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#16] @@ -102,24 +102,24 @@ ReadSchema: struct Input [1]: [s_store_sk#17] Condition : isnotnull(s_store_sk#17) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [s_store_sk#17] (17) BroadcastExchange Input [1]: [s_store_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#11] Right keys [1]: [s_store_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16, s_store_sk#17] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] Keys [2]: [i_manufact_id#5, d_qoy#16] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] @@ -130,10 +130,10 @@ Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Keys [2]: [i_manufact_id#5, d_qoy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] @@ -148,18 +148,18 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColum Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] (27) Window Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 7] +(29) Project [codegen id : 4] Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] @@ -171,7 +171,7 @@ Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -192,7 +192,7 @@ Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_qoy#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt index 0b6c2edaa0..5c7a26057d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject +- Project +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -25,17 +25,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/simplified.txt index 63c3e1a17a..d427ecd731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/simplified.txt @@ -1,53 +1,47 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id,d_qoy] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometNativeColumnarToRow + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + WholeStageCodegen (3) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_manufact_id,d_qoy] #2 + WholeStageCodegen (2) + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 + CometNativeColumnarToRow + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + ReusedExchange [d_date_sk,d_qoy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt index 5b68f4c00c..0a2e061781 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt index 0c123eb728..e0771ad655 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt index 5b68f4c00c..0a2e061781 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0c123eb728..e0771ad655 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/explain.txt index e362f08727..228692a53b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == TakeOrderedAndProject (61) +- * HashAggregate (60) - +- * CometColumnarToRow (59) + +- CometNativeColumnarToRow (59) +- CometColumnarExchange (58) +- * HashAggregate (57) +- * HashAggregate (56) - +- * CometColumnarToRow (55) + +- CometNativeColumnarToRow (55) +- CometColumnarExchange (54) +- * HashAggregate (53) +- * Project (52) @@ -16,7 +16,7 @@ TakeOrderedAndProject (61) : : +- * BroadcastHashJoin Inner BuildRight (41) : : :- * Project (35) : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * CometColumnarToRow (29) + : : : :- CometNativeColumnarToRow (29) : : : : +- CometHashAggregate (28) : : : : +- CometColumnarExchange (27) : : : : +- * HashAggregate (26) @@ -36,13 +36,13 @@ TakeOrderedAndProject (61) : : : : : : : +- * ColumnarToRow (6) : : : : : : : +- Scan parquet spark_catalog.default.web_sales (5) : : : : : : +- BroadcastExchange (14) - : : : : : : +- * CometColumnarToRow (13) + : : : : : : +- CometNativeColumnarToRow (13) : : : : : : +- CometProject (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- ReusedExchange (17) : : : : +- BroadcastExchange (23) - : : : : +- * CometColumnarToRow (22) + : : : : +- CometNativeColumnarToRow (22) : : : : +- CometFilter (21) : : : : +- CometNativeScan parquet spark_catalog.default.customer (20) : : : +- BroadcastExchange (33) @@ -50,12 +50,12 @@ TakeOrderedAndProject (61) : : : +- * ColumnarToRow (31) : : : +- Scan parquet spark_catalog.default.store_sales (30) : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) + : : +- CometNativeColumnarToRow (39) : : +- CometProject (38) : : +- CometFilter (37) : : +- CometNativeScan parquet spark_catalog.default.customer_address (36) : +- BroadcastExchange (47) - : +- * CometColumnarToRow (46) + : +- CometNativeColumnarToRow (46) : +- CometProject (45) : +- CometFilter (44) : +- CometNativeScan parquet spark_catalog.default.store (43) @@ -117,33 +117,33 @@ Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (static_ Input [3]: [i_item_sk#14, i_class#15, i_category#16] Arguments: [i_item_sk#14], [i_item_sk#14] -(13) CometColumnarToRow [codegen id : 3] +(13) CometNativeColumnarToRow Input [1]: [i_item_sk#14] (14) BroadcastExchange Input [1]: [i_item_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(15) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(16) Project [codegen id : 6] +(16) Project [codegen id : 3] Output [2]: [sold_date_sk#5, customer_sk#6] Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] (17) ReusedExchange [Reuses operator id: 66] Output [1]: [d_date_sk#17] -(18) BroadcastHashJoin [codegen id : 6] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sold_date_sk#5] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(19) Project [codegen id : 3] Output [1]: [customer_sk#6] Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] @@ -158,24 +158,24 @@ ReadSchema: struct Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [2]: [c_customer_sk#18, c_current_addr_sk#19] (23) BroadcastExchange Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 3] Left keys [1]: [customer_sk#6] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 3] Output [2]: [c_customer_sk#18, c_current_addr_sk#19] Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 3] Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] Functions: [] @@ -191,7 +191,7 @@ Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] Functions: [] -(29) CometColumnarToRow [codegen id : 11] +(29) CometNativeColumnarToRow Input [2]: [c_customer_sk#18, c_current_addr_sk#19] (30) Scan parquet spark_catalog.default.store_sales @@ -202,10 +202,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] +(31) ColumnarToRow [codegen id : 4] Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -(32) Filter [codegen id : 7] +(32) Filter [codegen id : 4] Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) @@ -213,13 +213,13 @@ Condition : isnotnull(ss_customer_sk#20) Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#18] Right keys [1]: [ss_customer_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 11] +(35) Project [codegen id : 5] Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] @@ -238,20 +238,20 @@ Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnot Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] Arguments: [ca_address_sk#24, ca_county#25, ca_state#27], [ca_address_sk#24, ca_county#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#26, 2)) AS ca_state#27] -(39) CometColumnarToRow [codegen id : 8] +(39) CometNativeColumnarToRow Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] (40) BroadcastExchange Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#19] Right keys [1]: [ca_address_sk#24] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(42) Project [codegen id : 5] Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27] Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#27] @@ -270,37 +270,37 @@ Condition : (isnotnull(s_county#28) AND isnotnull(static_invoke(CharVarcharCodeg Input [2]: [s_county#28, s_state#29] Arguments: [s_county#28, s_state#30], [s_county#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#29, 2)) AS s_state#30] -(46) CometColumnarToRow [codegen id : 9] +(46) CometNativeColumnarToRow Input [2]: [s_county#28, s_state#30] (47) BroadcastExchange Input [2]: [s_county#28, s_state#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 5] Left keys [2]: [ca_county#25, ca_state#27] Right keys [2]: [s_county#28, s_state#30] Join type: Inner Join condition: None -(49) Project [codegen id : 11] +(49) Project [codegen id : 5] Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27, s_county#28, s_state#30] (50) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#31] -(51) BroadcastHashJoin [codegen id : 11] +(51) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(52) Project [codegen id : 11] +(52) Project [codegen id : 5] Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#31] -(53) HashAggregate [codegen id : 11] +(53) HashAggregate [codegen id : 5] Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] Keys [1]: [c_customer_sk#18] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] @@ -311,17 +311,17 @@ Results [2]: [c_customer_sk#18, sum#33] Input [2]: [c_customer_sk#18, sum#33] Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(55) CometColumnarToRow [codegen id : 12] +(55) CometNativeColumnarToRow Input [2]: [c_customer_sk#18, sum#33] -(56) HashAggregate [codegen id : 12] +(56) HashAggregate [codegen id : 6] Input [2]: [c_customer_sk#18, sum#33] Keys [1]: [c_customer_sk#18] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#34] Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#34,17,2) / 50) as int) AS segment#35] -(57) HashAggregate [codegen id : 12] +(57) HashAggregate [codegen id : 6] Input [1]: [segment#35] Keys [1]: [segment#35] Functions [1]: [partial_count(1)] @@ -332,10 +332,10 @@ Results [2]: [segment#35, count#37] Input [2]: [segment#35, count#37] Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(59) CometColumnarToRow [codegen id : 13] +(59) CometNativeColumnarToRow Input [2]: [segment#35, count#37] -(60) HashAggregate [codegen id : 13] +(60) HashAggregate [codegen id : 7] Input [2]: [segment#35, count#37] Keys [1]: [segment#35] Functions [1]: [count(1)] @@ -350,7 +350,7 @@ Arguments: 100, [segment#35 ASC NULLS FIRST, num_customers#39 ASC NULLS FIRST], Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (66) -+- * CometColumnarToRow (65) ++- CometNativeColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometNativeScan parquet spark_catalog.default.date_dim (62) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 12 Input [3]: [d_date_sk#17, d_year#41, d_moy#42] Arguments: [d_date_sk#17], [d_date_sk#17] -(65) CometColumnarToRow [codegen id : 1] +(65) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (66) BroadcastExchange @@ -382,7 +382,7 @@ Subquery:2 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometNativeScan parquet spark_catalog.default.date_dim (67) @@ -403,7 +403,7 @@ Condition : (((isnotnull(d_month_seq#43) AND (d_month_seq#43 >= ReusedSubquery S Input [2]: [d_date_sk#31, d_month_seq#43] Arguments: [d_date_sk#31], [d_date_sk#31] -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [1]: [d_date_sk#31] (71) BroadcastExchange @@ -415,7 +415,7 @@ Subquery:4 Hosting operator id = 68 Hosting Expression = ReusedSubquery Subquery Subquery:5 Hosting operator id = 68 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#47] Subquery:6 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#44, [id=#46] -* CometColumnarToRow (78) +CometNativeColumnarToRow (78) +- CometHashAggregate (77) +- CometExchange (76) +- CometHashAggregate (75) @@ -453,11 +453,11 @@ Input [1]: [(d_month_seq + 1)#51] Keys [1]: [(d_month_seq + 1)#51] Functions: [] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [1]: [(d_month_seq + 1)#51] Subquery:7 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#45, [id=#47] -* CometColumnarToRow (85) +CometNativeColumnarToRow (85) +- CometHashAggregate (84) +- CometExchange (83) +- CometHashAggregate (82) @@ -495,7 +495,7 @@ Input [1]: [(d_month_seq + 3)#55] Keys [1]: [(d_month_seq + 3)#55] Functions: [] -(85) CometColumnarToRow [codegen id : 1] +(85) CometNativeColumnarToRow Input [1]: [(d_month_seq + 3)#55] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt index 51c5ebdbea..8da62eb377 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,17 +42,17 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange @@ -61,14 +61,14 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : :- ReusedSubquery : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : :- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -76,7 +76,7 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -84,24 +84,24 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter : :- ReusedSubquery : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim :- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -109,7 +109,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Subquery - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/simplified.txt index 6e6245b520..28767f39c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [segment] #1 - WholeStageCodegen (12) + WholeStageCodegen (6) HashAggregate [segment] [count,count] HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_sk] #2 - WholeStageCodegen (11) + WholeStageCodegen (5) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -20,11 +20,11 @@ TakeOrderedAndProject [segment,num_customers,segment_base] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [c_customer_sk,c_current_addr_sk] CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [c_customer_sk,c_current_addr_sk] Project [c_customer_sk,c_current_addr_sk] BroadcastHashJoin [customer_sk,c_customer_sk] @@ -42,12 +42,10 @@ TakeOrderedAndProject [segment,num_customers,segment_base] Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (2) Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] Filter [ws_item_sk,ws_bill_customer_sk] @@ -57,73 +55,59 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ReusedSubquery [d_date_sk] #1 InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #9 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #10 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #3 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #9 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #10 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] InputAdapter BroadcastExchange #11 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_county,ca_state] + CometFilter [ca_address_sk,ca_county,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] InputAdapter BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_county,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_county,s_state] + CometFilter [s_county,s_state] + CometNativeScan parquet spark_catalog.default.store [s_county,s_state] InputAdapter ReusedExchange [d_date_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt index 9eb7865391..dc1dee10d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt @@ -337,7 +337,7 @@ Input [3]: [segment#40, num_customers#42, segment_base#43] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) @@ -358,7 +358,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (63) BroadcastExchange @@ -369,7 +369,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (68) -+- * CometColumnarToRow (67) ++- CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) @@ -390,7 +390,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery S Input [2]: [d_date_sk#33, d_month_seq#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (68) BroadcastExchange @@ -402,7 +402,7 @@ Subquery:4 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] Subquery:6 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#35, [id=#37] -* CometColumnarToRow (75) +CometNativeColumnarToRow (75) +- CometHashAggregate (74) +- CometExchange (73) +- CometHashAggregate (72) @@ -440,11 +440,11 @@ Input [1]: [(d_month_seq + 1)#47] Keys [1]: [(d_month_seq + 1)#47] Functions: [] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [(d_month_seq + 1)#47] Subquery:7 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#36, [id=#38] -* CometColumnarToRow (82) +CometNativeColumnarToRow (82) +- CometHashAggregate (81) +- CometExchange (80) +- CometHashAggregate (79) @@ -482,7 +482,7 @@ Input [1]: [(d_month_seq + 3)#51] Keys [1]: [(d_month_seq + 3)#51] Functions: [] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [(d_month_seq + 3)#51] Subquery:8 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt index a49bbabd10..0e8e75c034 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,14 +53,14 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : :- ReusedSubquery : : : : +- ReusedSubquery : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : :- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -68,7 +68,7 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt index ce1acded44..7433373dcb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] @@ -57,34 +55,28 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #3 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 CometProject [ca_state] [ca_address_sk,ca_county,ca_state] CometFilter [ca_address_sk,ca_county,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt index 9eb7865391..dc1dee10d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt @@ -337,7 +337,7 @@ Input [3]: [segment#40, num_customers#42, segment_base#43] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) @@ -358,7 +358,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (63) BroadcastExchange @@ -369,7 +369,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (68) -+- * CometColumnarToRow (67) ++- CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) @@ -390,7 +390,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery S Input [2]: [d_date_sk#33, d_month_seq#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (68) BroadcastExchange @@ -402,7 +402,7 @@ Subquery:4 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] Subquery:6 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#35, [id=#37] -* CometColumnarToRow (75) +CometNativeColumnarToRow (75) +- CometHashAggregate (74) +- CometExchange (73) +- CometHashAggregate (72) @@ -440,11 +440,11 @@ Input [1]: [(d_month_seq + 1)#47] Keys [1]: [(d_month_seq + 1)#47] Functions: [] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [(d_month_seq + 1)#47] Subquery:7 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#36, [id=#38] -* CometColumnarToRow (82) +CometNativeColumnarToRow (82) +- CometHashAggregate (81) +- CometExchange (80) +- CometHashAggregate (79) @@ -482,7 +482,7 @@ Input [1]: [(d_month_seq + 3)#51] Keys [1]: [(d_month_seq + 3)#51] Functions: [] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [(d_month_seq + 3)#51] Subquery:8 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt index a49bbabd10..0e8e75c034 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,14 +53,14 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : :- ReusedSubquery : : : : +- ReusedSubquery : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : :- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -68,7 +68,7 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate 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 ce1acded44..7433373dcb 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 @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] @@ -57,34 +55,28 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #3 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 CometProject [ca_state] [ca_address_sk,ca_county,ca_state] CometFilter [ca_address_sk,ca_county,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/explain.txt index d2f7dbb904..3de39611eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [3]: [brand_id#13, brand#14, ext_price#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [3]: [brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/simplified.txt index a2adacc531..c31575cc36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_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 [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] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [brand_id,brand,ext_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 [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] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/explain.txt index dab8a2806c..8199e2d973 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (67) +- * HashAggregate (66) - +- * CometColumnarToRow (65) + +- CometNativeColumnarToRow (65) +- CometColumnarExchange (64) +- * HashAggregate (63) +- Union (62) :- * HashAggregate (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometColumnarExchange (27) : +- * HashAggregate (26) : +- * Project (25) @@ -20,12 +20,12 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometProject (21) : +- CometBroadcastHashJoin (20) : :- CometFilter (15) @@ -35,7 +35,7 @@ TakeOrderedAndProject (67) : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.item (16) :- * HashAggregate (45) - : +- * CometColumnarToRow (44) + : +- CometNativeColumnarToRow (44) : +- CometColumnarExchange (43) : +- * HashAggregate (42) : +- * Project (41) @@ -51,7 +51,7 @@ TakeOrderedAndProject (67) : : +- ReusedExchange (36) : +- ReusedExchange (39) +- * HashAggregate (61) - +- * CometColumnarToRow (60) + +- CometNativeColumnarToRow (60) +- CometColumnarExchange (59) +- * HashAggregate (58) +- * Project (57) @@ -76,23 +76,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] @@ -111,20 +111,20 @@ Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isno Input [2]: [ca_address_sk#7, ca_gmt_offset#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [ca_address_sk#7] (11) BroadcastExchange Input [1]: [ca_address_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] @@ -166,24 +166,24 @@ Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#10, Input [2]: [i_item_sk#9, i_item_id#10] Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#10, 16)) AS i_item_id#14] -(22) CometColumnarToRow [codegen id : 3] +(22) CometNativeColumnarToRow Input [2]: [i_item_sk#9, i_item_id#14] (23) BroadcastExchange Input [2]: [i_item_sk#9, i_item_id#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 4] +(24) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#9] Join type: Inner Join condition: None -(25) Project [codegen id : 4] +(25) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#3, i_item_id#14] Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] -(26) HashAggregate [codegen id : 4] +(26) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#14] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -194,10 +194,10 @@ Results [2]: [i_item_id#14, sum#16] Input [2]: [i_item_id#14, sum#16] Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [2]: [i_item_id#14, sum#16] -(29) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 2] Input [2]: [i_item_id#14, sum#16] Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -212,53 +212,53 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 9] +(31) ColumnarToRow [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -(32) Filter [codegen id : 9] +(32) Filter [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) (33) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#24] -(34) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#23] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(35) Project [codegen id : 9] +(35) Project [codegen id : 3] Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] (36) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#25] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_addr_sk#20] Right keys [1]: [ca_address_sk#25] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 3] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] (39) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#26, i_item_id#27] -(40) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#21] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(41) Project [codegen id : 3] Output [2]: [cs_ext_sales_price#22, i_item_id#27] Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 3] Input [2]: [cs_ext_sales_price#22, i_item_id#27] Keys [1]: [i_item_id#27] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] @@ -269,10 +269,10 @@ Results [2]: [i_item_id#27, sum#29] Input [2]: [i_item_id#27, sum#29] Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometColumnarToRow [codegen id : 10] +(44) CometNativeColumnarToRow Input [2]: [i_item_id#27, sum#29] -(45) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 4] Input [2]: [i_item_id#27, sum#29] Keys [1]: [i_item_id#27] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] @@ -287,53 +287,53 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 14] +(47) ColumnarToRow [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -(48) Filter [codegen id : 14] +(48) Filter [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) (49) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#36] -(50) BroadcastHashJoin [codegen id : 14] +(50) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#35] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(51) Project [codegen id : 14] +(51) Project [codegen id : 5] Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] (52) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#37] -(53) BroadcastHashJoin [codegen id : 14] +(53) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_bill_addr_sk#33] Right keys [1]: [ca_address_sk#37] Join type: Inner Join condition: None -(54) Project [codegen id : 14] +(54) Project [codegen id : 5] Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] (55) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#38, i_item_id#39] -(56) BroadcastHashJoin [codegen id : 14] +(56) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_item_sk#32] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(57) Project [codegen id : 5] Output [2]: [ws_ext_sales_price#34, i_item_id#39] Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] -(58) HashAggregate [codegen id : 14] +(58) HashAggregate [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#39] Keys [1]: [i_item_id#39] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] @@ -344,10 +344,10 @@ Results [2]: [i_item_id#39, sum#41] Input [2]: [i_item_id#39, sum#41] Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(60) CometColumnarToRow [codegen id : 15] +(60) CometNativeColumnarToRow Input [2]: [i_item_id#39, sum#41] -(61) HashAggregate [codegen id : 15] +(61) HashAggregate [codegen id : 6] Input [2]: [i_item_id#39, sum#41] Keys [1]: [i_item_id#39] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] @@ -356,7 +356,7 @@ Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34) (62) Union -(63) HashAggregate [codegen id : 16] +(63) HashAggregate [codegen id : 7] Input [2]: [i_item_id#18, total_sales#19] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(total_sales#19)] @@ -367,10 +367,10 @@ Results [3]: [i_item_id#18, sum#46, isEmpty#47] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(65) CometColumnarToRow [codegen id : 17] +(65) CometNativeColumnarToRow Input [3]: [i_item_id#18, sum#46, isEmpty#47] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 8] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Keys [1]: [i_item_id#18] Functions [1]: [sum(total_sales#19)] @@ -385,7 +385,7 @@ Arguments: 100, [total_sales#49 ASC NULLS FIRST], [i_item_id#18, total_sales#49] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometNativeScan parquet spark_catalog.default.date_dim (68) @@ -406,7 +406,7 @@ Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2 Input [3]: [d_date_sk#6, d_year#50, d_moy#51] Arguments: [d_date_sk#6], [d_date_sk#6] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (72) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt index 7a2832757d..ad055b48b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,22 +19,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -44,7 +44,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,17 +58,17 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -78,7 +78,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -92,17 +92,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/simplified.txt index 81d90bdfb2..4d9a45fc72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -27,41 +27,35 @@ TakeOrderedAndProject [total_sales,i_item_id] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (10) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #6 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_color] + CometNativeScan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -80,12 +74,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ReusedExchange [ca_address_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] BroadcastHashJoin [ws_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt index 16d7564ea5..78fb3411db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt index 19eca782cd..eb64094749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt index 16d7564ea5..78fb3411db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 19eca782cd..eb64094749 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/explain.txt index 9d35d600a6..ff75ec0f28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.call_center (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCode Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.catalog_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [2]: [cc_call_center_sk#14, cc_name#15] (17) BroadcastExchange Input [2]: [cc_call_center_sk#14, cc_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_call_center_sk#6] Right keys [1]: [cc_call_center_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] @@ -151,10 +151,10 @@ Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] (27) Window Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] (32) ReusedExchange [Reuses operator id: 21] Output [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] Keys [5]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(cs_sales_price#29))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#23, i_brand#24, cc_name#25, 5), ENSURE_RE Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, cc_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] (38) Window Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#23, i_brand#24, cc_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#23, i_brand#24, cc_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [5]: [i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] Input [7]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] @@ -228,13 +228,13 @@ Input [7]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sale Input [5]: [i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#23, i_brand#24, cc_name#25, (rn#31 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#30] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] @@ -245,14 +245,14 @@ Output [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sal Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] Arguments: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, cc_name#34 ASC NULLS FIRST, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] (46) Window Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] Arguments: [rank(d_year#35, d_moy#36) windowspecdefinition(i_category#32, i_brand#33, cc_name#34, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, cc_name#34], [d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [5]: [i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] Input [7]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37, rn#38] @@ -260,13 +260,13 @@ Input [7]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sale Input [5]: [i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#32, i_brand#33, cc_name#34, (rn#38 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#30 AS psum#39, sum_sales#37 AS nsum#40] Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#30, i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt index 65c66a7da8..66c510aaa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/simplified.txt index 4f9ac35f4d..c085ecfb51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [cs_item_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [cc_call_center_sk,cc_name] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt index 7fec2cf2e8..8406438b9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt index b854e818db..2f01d9b13f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt index 7fec2cf2e8..8406438b9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 b854e818db..2f01d9b13f 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/explain.txt index 6ea099c1da..f758f90e6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (53) : +- * BroadcastHashJoin Inner BuildRight (34) : :- * Filter (18) : : +- * HashAggregate (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- * HashAggregate (14) : : +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (53) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -25,7 +25,7 @@ TakeOrderedAndProject (53) : +- BroadcastExchange (33) : +- * Filter (32) : +- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * Project (27) @@ -40,7 +40,7 @@ TakeOrderedAndProject (53) +- BroadcastExchange (50) +- * Filter (49) +- * HashAggregate (48) - +- * CometColumnarToRow (47) + +- CometNativeColumnarToRow (47) +- CometColumnarExchange (46) +- * HashAggregate (45) +- * Project (44) @@ -62,10 +62,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) @@ -84,37 +84,37 @@ Condition : (isnotnull(i_item_sk#5) AND isnotnull(static_invoke(CharVarcharCodeg Input [2]: [i_item_sk#5, i_item_id#6] Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#7] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [i_item_sk#5, i_item_id#7] (8) BroadcastExchange Input [2]: [i_item_sk#5, i_item_id#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] (11) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#8] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#2, i_item_id#7] Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#2, i_item_id#7] Keys [1]: [i_item_id#7] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -125,17 +125,17 @@ Results [2]: [i_item_id#7, sum#10] Input [2]: [i_item_id#7, sum#10] Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 12] +(16) CometNativeColumnarToRow Input [2]: [i_item_id#7, sum#10] -(17) HashAggregate [codegen id : 12] +(17) HashAggregate [codegen id : 6] Input [2]: [i_item_id#7, sum#10] Keys [1]: [i_item_id#7] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] Results [2]: [i_item_id#7 AS item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS ss_item_rev#13] -(18) Filter [codegen id : 12] +(18) Filter [codegen id : 6] Input [2]: [item_id#12, ss_item_rev#13] Condition : isnotnull(ss_item_rev#13) @@ -147,40 +147,40 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 2] Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -(21) Filter [codegen id : 6] +(21) Filter [codegen id : 2] Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] Condition : isnotnull(cs_item_sk#14) (22) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#17, i_item_id#18] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_item_sk#14] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(24) Project [codegen id : 2] Output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#18] Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#17, i_item_id#18] (25) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#19] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 2] Output [2]: [cs_ext_sales_price#15, i_item_id#18] Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#18, d_date_sk#19] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [2]: [cs_ext_sales_price#15, i_item_id#18] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] @@ -191,17 +191,17 @@ Results [2]: [i_item_id#18, sum#21] Input [2]: [i_item_id#18, sum#21] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [2]: [i_item_id#18, sum#21] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [2]: [i_item_id#18, sum#21] Keys [1]: [i_item_id#18] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#15))#22] Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#15))#22,17,2) AS cs_item_rev#24] -(32) Filter [codegen id : 7] +(32) Filter [codegen id : 3] Input [2]: [item_id#23, cs_item_rev#24] Condition : isnotnull(cs_item_rev#24) @@ -209,13 +209,13 @@ Condition : isnotnull(cs_item_rev#24) Input [2]: [item_id#23, cs_item_rev#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 12] +(34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#23] Join type: Inner Join condition: ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) -(35) Project [codegen id : 12] +(35) Project [codegen id : 6] Output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#24] Input [4]: [item_id#12, ss_item_rev#13, item_id#23, cs_item_rev#24] @@ -227,40 +227,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 4] Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -(38) Filter [codegen id : 10] +(38) Filter [codegen id : 4] Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] Condition : isnotnull(ws_item_sk#25) (39) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#28, i_item_id#29] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#25] Right keys [1]: [i_item_sk#28] Join type: Inner Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 4] Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#29] Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#28, i_item_id#29] (42) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#30] -(43) BroadcastHashJoin [codegen id : 10] +(43) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#27] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(44) Project [codegen id : 10] +(44) Project [codegen id : 4] Output [2]: [ws_ext_sales_price#26, i_item_id#29] Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#29, d_date_sk#30] -(45) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 4] Input [2]: [ws_ext_sales_price#26, i_item_id#29] Keys [1]: [i_item_id#29] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] @@ -271,17 +271,17 @@ Results [2]: [i_item_id#29, sum#32] Input [2]: [i_item_id#29, sum#32] Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(47) CometColumnarToRow [codegen id : 11] +(47) CometNativeColumnarToRow Input [2]: [i_item_id#29, sum#32] -(48) HashAggregate [codegen id : 11] +(48) HashAggregate [codegen id : 5] Input [2]: [i_item_id#29, sum#32] Keys [1]: [i_item_id#29] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#33] Results [2]: [i_item_id#29 AS item_id#34, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#33,17,2) AS ws_item_rev#35] -(49) Filter [codegen id : 11] +(49) Filter [codegen id : 5] Input [2]: [item_id#34, ws_item_rev#35] Condition : isnotnull(ws_item_rev#35) @@ -289,13 +289,13 @@ Condition : isnotnull(ws_item_rev#35) Input [2]: [item_id#34, ws_item_rev#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(51) BroadcastHashJoin [codegen id : 12] +(51) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#34] Join type: Inner Join condition: ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#35)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) -(52) Project [codegen id : 12] +(52) Project [codegen id : 6] Output [8]: [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ss_dev#36, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS cs_dev#37, ws_item_rev#35, (((ws_item_rev#35 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ws_dev#38, (((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35) / 3) AS average#39] Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#24, item_id#34, ws_item_rev#35] @@ -307,7 +307,7 @@ Arguments: 100, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometFilter (55) @@ -357,7 +357,7 @@ Arguments: [d_date#40], [d_date#41], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#40] Arguments: [d_date_sk#8], [d_date_sk#8] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (63) BroadcastExchange @@ -367,7 +367,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] Subquery:3 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#43, [id=#44] -* CometColumnarToRow (67) +CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometNativeScan parquet spark_catalog.default.date_dim (64) @@ -388,7 +388,7 @@ Condition : (isnotnull(d_date#45) AND (d_date#45 = 2000-01-03)) Input [2]: [d_date#45, d_week_seq#46] Arguments: [d_week_seq#46], [d_week_seq#46] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_week_seq#46] Subquery:4 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt index 787f30c437..dd95b1b293 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -28,17 +28,17 @@ TakeOrderedAndProject : : : : : +- ReusedSubquery : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter @@ -49,14 +49,14 @@ TakeOrderedAndProject : : : +- ReusedSubquery : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -68,12 +68,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -84,14 +84,14 @@ TakeOrderedAndProject : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -103,12 +103,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter @@ -119,7 +119,7 @@ TakeOrderedAndProject : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim +- Subquery - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/simplified.txt index fd841da602..5361566804 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/simplified.txt @@ -1,15 +1,15 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (12) + WholeStageCodegen (6) Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] Project [item_id,ss_item_rev,cs_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] Filter [ss_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -21,44 +21,38 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (3) Filter [cs_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -75,13 +69,13 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) + WholeStageCodegen (5) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (10) + WholeStageCodegen (4) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt index 3c368afc65..0c814275a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt @@ -292,7 +292,7 @@ Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometFilter (54) @@ -342,7 +342,7 @@ Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (62) BroadcastExchange @@ -352,7 +352,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] Subquery:3 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* CometColumnarToRow (66) +CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -373,7 +373,7 @@ Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) Input [2]: [d_date#41, d_week_seq#42] Arguments: [d_week_seq#42], [d_week_seq#42] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_week_seq#42] Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt index b4255cb7a1..926061b466 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : +- ReusedSubquery : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -46,7 +46,7 @@ CometColumnarToRow : : : +- ReusedSubquery : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -77,7 +77,7 @@ CometColumnarToRow : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -108,7 +108,7 @@ CometColumnarToRow : +- ReusedSubquery +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Subquery - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt index 9d8bcdb40a..a7d7977c93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt @@ -18,25 +18,21 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt index 3c368afc65..0c814275a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt @@ -292,7 +292,7 @@ Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometFilter (54) @@ -342,7 +342,7 @@ Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (62) BroadcastExchange @@ -352,7 +352,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] Subquery:3 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* CometColumnarToRow (66) +CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -373,7 +373,7 @@ Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) Input [2]: [d_date#41, d_week_seq#42] Arguments: [d_week_seq#42], [d_week_seq#42] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_week_seq#42] Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt index b4255cb7a1..926061b466 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : +- ReusedSubquery : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -46,7 +46,7 @@ CometColumnarToRow : : : +- ReusedSubquery : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -77,7 +77,7 @@ CometColumnarToRow : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -108,7 +108,7 @@ CometColumnarToRow : +- ReusedSubquery +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Subquery - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 9d8bcdb40a..a7d7977c93 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 @@ -18,25 +18,21 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/explain.txt index 9270d963a1..34735e47cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (42) +CometNativeColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -234,6 +234,6 @@ Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_ Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#70,(mon_sales1 / mon_sales2)#71,(tue_sales1 / tue_sales2)#72,(wed_sales1 / wed_sales2)#73,(thu_sales1 / thu_sales2)#74,(fri_sales1 / fri_sales2)#75,(sat_sales1 / sat_sales2)#76]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] -(42) CometColumnarToRow [codegen id : 1] +(42) CometNativeColumnarToRow Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/extended.txt index bfd467b72b..e23616d099 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/simplified.txt index 72823b14c1..16c54015b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/simplified.txt @@ -1,44 +1,42 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [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_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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [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] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] +CometNativeColumnarToRow + CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] + CometHashAggregate [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_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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] + CometHashAggregate [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] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [d_week_seq] #7 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/explain.txt index 48c6b7db44..abcaeb7ceb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (40) +- * Filter (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * Project (34) @@ -11,7 +11,7 @@ TakeOrderedAndProject (40) : +- * BroadcastHashJoin Inner BuildRight (17) : :- * Project (15) : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) + : : :- CometNativeColumnarToRow (9) : : : +- CometProject (8) : : : +- CometBroadcastHashJoin (7) : : : :- CometProject (3) @@ -26,7 +26,7 @@ TakeOrderedAndProject (40) : : +- Scan parquet spark_catalog.default.store_sales (10) : +- ReusedExchange (16) +- BroadcastExchange (32) - +- * CometColumnarToRow (31) + +- CometNativeColumnarToRow (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (20) @@ -80,7 +80,7 @@ Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(9) CometColumnarToRow [codegen id : 4] +(9) CometNativeColumnarToRow Input [2]: [ca_state#3, c_customer_sk#4] (10) Scan parquet spark_catalog.default.store_sales @@ -102,26 +102,26 @@ Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#4] Right keys [1]: [ss_customer_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] (16) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 2] Output [2]: [ca_state#3, ss_item_sk#6] Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] @@ -181,24 +181,24 @@ Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#13, Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] Arguments: [i_item_sk#11], [i_item_sk#11] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [i_item_sk#11] (32) BroadcastExchange Input [1]: [i_item_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 4] +(33) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#6] Right keys [1]: [i_item_sk#11] Join type: Inner Join condition: None -(34) Project [codegen id : 4] +(34) Project [codegen id : 2] Output [1]: [ca_state#3] Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] -(35) HashAggregate [codegen id : 4] +(35) HashAggregate [codegen id : 2] Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] @@ -209,17 +209,17 @@ Results [2]: [ca_state#3, count#21] Input [2]: [ca_state#3, count#21] Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometColumnarToRow [codegen id : 5] +(37) CometNativeColumnarToRow Input [2]: [ca_state#3, count#21] -(38) HashAggregate [codegen id : 5] +(38) HashAggregate [codegen id : 3] Input [2]: [ca_state#3, count#21] Keys [1]: [ca_state#3] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] Results [2]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24] -(39) Filter [codegen id : 5] +(39) Filter [codegen id : 3] Input [2]: [state#23, cnt#24] Condition : (cnt#24 >= 10) @@ -231,7 +231,7 @@ Arguments: 100, [cnt#24 ASC NULLS FIRST], [state#23, cnt#24] Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = ReusedSubquery Sub Input [2]: [d_date_sk#10, d_month_seq#25] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -262,7 +262,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#26, [id=#27] Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -300,7 +300,7 @@ Input [1]: [d_month_seq#28] Keys [1]: [d_month_seq#28] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt index ee120ac920..44f11f06fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometProject @@ -25,13 +25,13 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -39,13 +39,13 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -53,7 +53,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/simplified.txt index 6b96848c69..95f9c38694 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) + WholeStageCodegen (3) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -13,8 +13,8 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ca_state,ss_item_sk,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ca_state,c_customer_sk] CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] CometProject [ca_state] [ca_address_sk,ca_state] @@ -32,39 +32,33 @@ TakeOrderedAndProject [cnt,state] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_category,i_current_price] - CometFilter [i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #7 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #8 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_category,i_current_price] + CometFilter [i_current_price,i_category] + CometNativeScan 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/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/explain.txt index 7871dbc31e..554d6f28b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#24, cnt#25] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Sub Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -262,7 +262,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -300,7 +300,7 @@ Input [1]: [d_month_seq#26] Keys [1]: [d_month_seq#26] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#26] Subquery:4 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt index 2d049f149f..4bb14d3c19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt @@ -23,13 +23,13 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/simplified.txt index 010720bc0e..76ccfa2abe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/simplified.txt @@ -25,23 +25,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt index 7871dbc31e..554d6f28b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#24, cnt#25] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Sub Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -262,7 +262,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -300,7 +300,7 @@ Input [1]: [d_month_seq#26] Keys [1]: [d_month_seq#26] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#26] Subquery:4 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/extended.txt index 2d049f149f..4bb14d3c19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/extended.txt @@ -23,13 +23,13 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate 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 010720bc0e..76ccfa2abe 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 @@ -25,23 +25,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/explain.txt index 38d1c84a80..1299ebb40d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (67) +- * HashAggregate (66) - +- * CometColumnarToRow (65) + +- CometNativeColumnarToRow (65) +- CometColumnarExchange (64) +- * HashAggregate (63) +- Union (62) :- * HashAggregate (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometColumnarExchange (27) : +- * HashAggregate (26) : +- * Project (25) @@ -20,12 +20,12 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometProject (21) : +- CometBroadcastHashJoin (20) : :- CometFilter (15) @@ -35,7 +35,7 @@ TakeOrderedAndProject (67) : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.item (16) :- * HashAggregate (45) - : +- * CometColumnarToRow (44) + : +- CometNativeColumnarToRow (44) : +- CometColumnarExchange (43) : +- * HashAggregate (42) : +- * Project (41) @@ -51,7 +51,7 @@ TakeOrderedAndProject (67) : : +- ReusedExchange (36) : +- ReusedExchange (39) +- * HashAggregate (61) - +- * CometColumnarToRow (60) + +- CometNativeColumnarToRow (60) +- CometColumnarExchange (59) +- * HashAggregate (58) +- * Project (57) @@ -76,23 +76,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] @@ -111,20 +111,20 @@ Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isno Input [2]: [ca_address_sk#7, ca_gmt_offset#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [ca_address_sk#7] (11) BroadcastExchange Input [1]: [ca_address_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] @@ -167,24 +167,24 @@ Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#10, Input [2]: [i_item_sk#9, i_item_id#10] Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#10, 16)) AS i_item_id#14] -(22) CometColumnarToRow [codegen id : 3] +(22) CometNativeColumnarToRow Input [2]: [i_item_sk#9, i_item_id#14] (23) BroadcastExchange Input [2]: [i_item_sk#9, i_item_id#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 4] +(24) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#9] Join type: Inner Join condition: None -(25) Project [codegen id : 4] +(25) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#3, i_item_id#14] Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] -(26) HashAggregate [codegen id : 4] +(26) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#14] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -195,10 +195,10 @@ Results [2]: [i_item_id#14, sum#16] Input [2]: [i_item_id#14, sum#16] Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [2]: [i_item_id#14, sum#16] -(29) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 2] Input [2]: [i_item_id#14, sum#16] Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -213,53 +213,53 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 9] +(31) ColumnarToRow [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -(32) Filter [codegen id : 9] +(32) Filter [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) (33) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#24] -(34) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#23] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(35) Project [codegen id : 9] +(35) Project [codegen id : 3] Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] (36) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#25] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_addr_sk#20] Right keys [1]: [ca_address_sk#25] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 3] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] (39) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#26, i_item_id#27] -(40) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#21] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(41) Project [codegen id : 3] Output [2]: [cs_ext_sales_price#22, i_item_id#27] Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 3] Input [2]: [cs_ext_sales_price#22, i_item_id#27] Keys [1]: [i_item_id#27] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] @@ -270,10 +270,10 @@ Results [2]: [i_item_id#27, sum#29] Input [2]: [i_item_id#27, sum#29] Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometColumnarToRow [codegen id : 10] +(44) CometNativeColumnarToRow Input [2]: [i_item_id#27, sum#29] -(45) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 4] Input [2]: [i_item_id#27, sum#29] Keys [1]: [i_item_id#27] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] @@ -288,53 +288,53 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 14] +(47) ColumnarToRow [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -(48) Filter [codegen id : 14] +(48) Filter [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) (49) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#36] -(50) BroadcastHashJoin [codegen id : 14] +(50) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#35] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(51) Project [codegen id : 14] +(51) Project [codegen id : 5] Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] (52) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#37] -(53) BroadcastHashJoin [codegen id : 14] +(53) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_bill_addr_sk#33] Right keys [1]: [ca_address_sk#37] Join type: Inner Join condition: None -(54) Project [codegen id : 14] +(54) Project [codegen id : 5] Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] (55) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#38, i_item_id#39] -(56) BroadcastHashJoin [codegen id : 14] +(56) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_item_sk#32] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(57) Project [codegen id : 5] Output [2]: [ws_ext_sales_price#34, i_item_id#39] Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] -(58) HashAggregate [codegen id : 14] +(58) HashAggregate [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#39] Keys [1]: [i_item_id#39] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] @@ -345,10 +345,10 @@ Results [2]: [i_item_id#39, sum#41] Input [2]: [i_item_id#39, sum#41] Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(60) CometColumnarToRow [codegen id : 15] +(60) CometNativeColumnarToRow Input [2]: [i_item_id#39, sum#41] -(61) HashAggregate [codegen id : 15] +(61) HashAggregate [codegen id : 6] Input [2]: [i_item_id#39, sum#41] Keys [1]: [i_item_id#39] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] @@ -357,7 +357,7 @@ Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34) (62) Union -(63) HashAggregate [codegen id : 16] +(63) HashAggregate [codegen id : 7] Input [2]: [i_item_id#18, total_sales#19] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(total_sales#19)] @@ -368,10 +368,10 @@ Results [3]: [i_item_id#18, sum#46, isEmpty#47] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(65) CometColumnarToRow [codegen id : 17] +(65) CometNativeColumnarToRow Input [3]: [i_item_id#18, sum#46, isEmpty#47] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 8] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Keys [1]: [i_item_id#18] Functions [1]: [sum(total_sales#19)] @@ -386,7 +386,7 @@ Arguments: 100, [i_item_id#18 ASC NULLS FIRST, total_sales#49 ASC NULLS FIRST], Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometNativeScan parquet spark_catalog.default.date_dim (68) @@ -407,7 +407,7 @@ Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 1 Input [3]: [d_date_sk#6, d_year#50, d_moy#51] Arguments: [d_date_sk#6], [d_date_sk#6] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (72) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt index 7a2832757d..ad055b48b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,22 +19,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -44,7 +44,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,17 +58,17 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -78,7 +78,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -92,17 +92,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/simplified.txt index 4c05038b34..56a5af7cf7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -27,41 +27,35 @@ TakeOrderedAndProject [i_item_id,total_sales] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (10) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #6 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -80,12 +74,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ReusedExchange [ca_address_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] BroadcastHashJoin [ws_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt index 1777ebee23..409e3e9ec1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt @@ -351,7 +351,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -372,7 +372,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt index 086da3b776..92a33e24ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt index 1777ebee23..409e3e9ec1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt @@ -351,7 +351,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -372,7 +372,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 086da3b776..92a33e24ce 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/explain.txt index 7eae15cdc2..ea507346af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/explain.txt @@ -2,7 +2,7 @@ * Project (69) +- * BroadcastNestedLoopJoin Inner BuildRight (68) :- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -21,33 +21,33 @@ : : : : : : : +- * ColumnarToRow (2) : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) + : : : : : : +- CometNativeColumnarToRow (7) : : : : : : +- CometProject (6) : : : : : : +- CometFilter (5) : : : : : : +- CometNativeScan parquet spark_catalog.default.store (4) : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) + : : : : : +- CometNativeColumnarToRow (14) : : : : : +- CometProject (13) : : : : : +- CometFilter (12) : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (11) : : : : +- ReusedExchange (18) : : : +- BroadcastExchange (24) - : : : +- * CometColumnarToRow (23) + : : : +- CometNativeColumnarToRow (23) : : : +- CometFilter (22) : : : +- CometNativeScan parquet spark_catalog.default.customer (21) : : +- BroadcastExchange (31) - : : +- * CometColumnarToRow (30) + : : +- CometNativeColumnarToRow (30) : : +- CometProject (29) : : +- CometFilter (28) : : +- CometNativeScan parquet spark_catalog.default.customer_address (27) : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometProject (36) : +- CometFilter (35) : +- CometNativeScan parquet spark_catalog.default.item (34) +- BroadcastExchange (67) +- * HashAggregate (66) - +- * CometColumnarToRow (65) + +- CometNativeColumnarToRow (65) +- CometColumnarExchange (64) +- * HashAggregate (63) +- * Project (62) @@ -78,10 +78,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) @@ -100,20 +100,20 @@ Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotn Input [2]: [s_store_sk#8, s_gmt_offset#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (8) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 1] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] @@ -132,33 +132,33 @@ Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_ Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] Arguments: [p_promo_sk#10], [p_promo_sk#10] -(14) CometColumnarToRow [codegen id : 2] +(14) CometNativeColumnarToRow Input [1]: [p_promo_sk#10] (15) BroadcastExchange Input [1]: [p_promo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 7] +(16) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_promo_sk#4] Right keys [1]: [p_promo_sk#10] Join type: Inner Join condition: None -(17) Project [codegen id : 7] +(17) Project [codegen id : 1] Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] (18) ReusedExchange [Reuses operator id: 74] Output [1]: [d_date_sk#14] -(19) BroadcastHashJoin [codegen id : 7] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 7] +(20) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] @@ -173,20 +173,20 @@ ReadSchema: struct Input [2]: [c_customer_sk#15, c_current_addr_sk#16] Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [c_customer_sk#15, c_current_addr_sk#16] (24) BroadcastExchange Input [2]: [c_customer_sk#15, c_current_addr_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 7] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 7] +(26) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#15, c_current_addr_sk#16] @@ -205,20 +205,20 @@ Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -5.00)) AND is Input [2]: [ca_address_sk#17, ca_gmt_offset#18] Arguments: [ca_address_sk#17], [ca_address_sk#17] -(30) CometColumnarToRow [codegen id : 5] +(30) CometNativeColumnarToRow Input [1]: [ca_address_sk#17] (31) BroadcastExchange Input [1]: [ca_address_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#16] Right keys [1]: [ca_address_sk#17] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16, ca_address_sk#17] @@ -237,24 +237,24 @@ Condition : ((isnotnull(i_category#20) AND (static_invoke(CharVarcharCodegenUtil Input [2]: [i_item_sk#19, i_category#20] Arguments: [i_item_sk#19], [i_item_sk#19] -(37) CometColumnarToRow [codegen id : 6] +(37) CometNativeColumnarToRow Input [1]: [i_item_sk#19] (38) BroadcastExchange Input [1]: [i_item_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 7] +(39) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#19] Join type: Inner Join condition: None -(40) Project [codegen id : 7] +(40) Project [codegen id : 1] Output [1]: [ss_ext_sales_price#5] Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#19] -(41) HashAggregate [codegen id : 7] +(41) HashAggregate [codegen id : 1] Input [1]: [ss_ext_sales_price#5] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] @@ -265,10 +265,10 @@ Results [1]: [sum#22] Input [1]: [sum#22] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 15] +(43) CometNativeColumnarToRow Input [1]: [sum#22] -(44) HashAggregate [codegen id : 15] +(44) HashAggregate [codegen id : 4] Input [1]: [sum#22] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] @@ -283,79 +283,79 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 13] +(46) ColumnarToRow [codegen id : 2] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -(47) Filter [codegen id : 13] +(47) Filter [codegen id : 2] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [s_store_sk#30] -(49) BroadcastHashJoin [codegen id : 13] +(49) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#27] Right keys [1]: [s_store_sk#30] Join type: Inner Join condition: None -(50) Project [codegen id : 13] +(50) Project [codegen id : 2] Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#30] (51) ReusedExchange [Reuses operator id: 74] Output [1]: [d_date_sk#31] -(52) BroadcastHashJoin [codegen id : 13] +(52) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#29] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(53) Project [codegen id : 13] +(53) Project [codegen id : 2] Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] (54) ReusedExchange [Reuses operator id: 24] Output [2]: [c_customer_sk#32, c_current_addr_sk#33] -(55) BroadcastHashJoin [codegen id : 13] +(55) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(56) Project [codegen id : 13] +(56) Project [codegen id : 2] Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#32, c_current_addr_sk#33] (57) ReusedExchange [Reuses operator id: 31] Output [1]: [ca_address_sk#34] -(58) BroadcastHashJoin [codegen id : 13] +(58) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_current_addr_sk#33] Right keys [1]: [ca_address_sk#34] Join type: Inner Join condition: None -(59) Project [codegen id : 13] +(59) Project [codegen id : 2] Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33, ca_address_sk#34] (60) ReusedExchange [Reuses operator id: 38] Output [1]: [i_item_sk#35] -(61) BroadcastHashJoin [codegen id : 13] +(61) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#25] Right keys [1]: [i_item_sk#35] Join type: Inner Join condition: None -(62) Project [codegen id : 13] +(62) Project [codegen id : 2] Output [1]: [ss_ext_sales_price#28] Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#35] -(63) HashAggregate [codegen id : 13] +(63) HashAggregate [codegen id : 2] Input [1]: [ss_ext_sales_price#28] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] @@ -366,10 +366,10 @@ Results [1]: [sum#37] Input [1]: [sum#37] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(65) CometColumnarToRow [codegen id : 14] +(65) CometNativeColumnarToRow Input [1]: [sum#37] -(66) HashAggregate [codegen id : 14] +(66) HashAggregate [codegen id : 3] Input [1]: [sum#37] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] @@ -380,11 +380,11 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#38,17,2) AS Input [1]: [total#39] Arguments: IdentityBroadcastMode, [plan_id=8] -(68) BroadcastNestedLoopJoin [codegen id : 15] +(68) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(69) Project [codegen id : 15] +(69) Project [codegen id : 4] Output [3]: [promotions#24, total#39, ((cast(promotions#24 as decimal(15,4)) / cast(total#39 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] Input [2]: [promotions#24, total#39] @@ -392,7 +392,7 @@ Input [2]: [promotions#24, total#39] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometProject (72) +- CometFilter (71) +- CometNativeScan parquet spark_catalog.default.date_dim (70) @@ -413,7 +413,7 @@ Condition : ((((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1 Input [3]: [d_date_sk#14, d_year#41, d_moy#42] Arguments: [d_date_sk#14], [d_date_sk#14] -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (74) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt index 0443f5fc55..025280d4ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt @@ -1,7 +1,7 @@ Project +- BroadcastNestedLoopJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,42 +21,42 @@ Project : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.promotion : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -74,26 +74,26 @@ Project : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/simplified.txt index f5bacac9c6..92507e130e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ -WholeStageCodegen (15) +WholeStageCodegen (4) Project [promotions,total] BroadcastNestedLoopJoin HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -25,61 +25,49 @@ WholeStageCodegen (15) Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometNativeColumnarToRow + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #8 - WholeStageCodegen (14) + WholeStageCodegen (3) HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (13) + WholeStageCodegen (2) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt index 9465cc9c1d..0526e8aa85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt index 7bae7d5724..f359e64d7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ Project : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt index 6ebe25b753..1709804080 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ WholeStageCodegen (2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] CometFilter [s_store_sk,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt index 9465cc9c1d..0526e8aa85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt index 7bae7d5724..f359e64d7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt @@ -20,7 +20,7 @@ Project : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6ebe25b753..1709804080 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 @@ -22,12 +22,10 @@ WholeStageCodegen (2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] CometFilter [s_store_sk,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/explain.txt index 9c710b15fb..630abc628f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (29) +CometNativeColumnarToRow (29) +- CometTakeOrderedAndProject (28) +- CometHashAggregate (27) +- CometExchange (26) @@ -163,6 +163,6 @@ Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt index 52d7828b06..dc88244743 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/simplified.txt index de80c17349..1407717d50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/simplified.txt @@ -1,31 +1,29 @@ -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 [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 [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] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +CometNativeColumnarToRow + 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 [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 [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] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometProject [sm_type] [sm_ship_mode_sk,sm_type] + CometFilter [sm_ship_mode_sk,sm_type] + CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/explain.txt index 03bd5bf65c..0b6e9b7dce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (30) +- * Project (29) +- * Filter (28) +- Window (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -16,7 +16,7 @@ TakeOrderedAndProject (30) : +- * BroadcastHashJoin Inner BuildRight (12) : :- * Project (10) : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject (30) : : +- Scan parquet spark_catalog.default.store_sales (5) : +- ReusedExchange (11) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -46,7 +46,7 @@ Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [2]: [i_item_sk#1, i_manager_id#5] (5) Scan parquet spark_catalog.default.store_sales @@ -68,26 +68,26 @@ Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] (11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#15, d_moy#16] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#13] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16] Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#16] @@ -102,24 +102,24 @@ ReadSchema: struct Input [1]: [s_store_sk#17] Condition : isnotnull(s_store_sk#17) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [s_store_sk#17] (17) BroadcastExchange Input [1]: [s_store_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#11] Right keys [1]: [s_store_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16, s_store_sk#17] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] Keys [2]: [i_manager_id#5, d_moy#16] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] @@ -130,10 +130,10 @@ Results [3]: [i_manager_id#5, d_moy#16, sum#19] Input [3]: [i_manager_id#5, d_moy#16, sum#19] Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [3]: [i_manager_id#5, d_moy#16, sum#19] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [3]: [i_manager_id#5, d_moy#16, sum#19] Keys [2]: [i_manager_id#5, d_moy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] @@ -148,18 +148,18 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumn Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] (27) Window Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 7] +(29) Project [codegen id : 4] Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] @@ -171,7 +171,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -192,7 +192,7 @@ Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_moy#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt index 0b6c2edaa0..5c7a26057d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject +- Project +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -25,17 +25,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/simplified.txt index 2933149226..7bb18e1026 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/simplified.txt @@ -1,53 +1,47 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manager_id,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometNativeColumnarToRow + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + WholeStageCodegen (3) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_manager_id,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt index 608bf20b3e..84811184c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt index 02166879a9..2f7b6145a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt index 608bf20b3e..84811184c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 02166879a9..2f7b6145a3 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/explain.txt index ff9f485dd9..b35e3ae589 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (180) +CometNativeColumnarToRow (180) +- CometSort (179) +- CometExchange (178) +- CometProject (177) @@ -48,7 +48,7 @@ : : : : : : : : : : : : : : : : : : +- * Filter (3) : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (8) : : : : : : : : : : : : : : : : : +- CometProject (7) : : : : : : : : : : : : : : : : : +- CometFilter (6) : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) @@ -153,7 +153,7 @@ : : : : : : : : : : : : : : : : : +- * Filter (108) : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (113) : : : : : : : : : : : : : : : : +- CometProject (112) : : : : : : : : : : : : : : : : +- CometFilter (111) : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) @@ -215,7 +215,7 @@ Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) CometColumnarToRow +(8) CometNativeColumnarToRow Input [2]: [sr_item_sk#14, sr_ticket_number#15] (9) BroadcastHashJoin [codegen id : 2] @@ -694,7 +694,7 @@ Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] -(113) CometColumnarToRow +(113) CometNativeColumnarToRow Input [2]: [sr_item_sk#111, sr_ticket_number#112] (114) BroadcastHashJoin [codegen id : 4] @@ -971,14 +971,14 @@ Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC 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#170, s2#171, s3#172, syear#168, cnt#169] 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#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST] -(180) CometColumnarToRow [codegen id : 5] +(180) CometNativeColumnarToRow 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#170, s2#171, s3#172, syear#168, cnt#169] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometNativeScan parquet spark_catalog.default.date_dim (181) @@ -994,7 +994,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (184) BroadcastExchange @@ -1003,7 +1003,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (188) -+- * CometColumnarToRow (187) ++- CometNativeColumnarToRow (187) +- CometFilter (186) +- CometNativeScan parquet spark_catalog.default.date_dim (185) @@ -1019,7 +1019,7 @@ ReadSchema: struct Input [2]: [d_date_sk#124, d_year#125] Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(187) CometColumnarToRow [codegen id : 1] +(187) CometNativeColumnarToRow Input [2]: [d_date_sk#124, d_year#125] (188) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt index 6e3f37148b..98db64e5b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -49,10 +49,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -171,10 +171,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/simplified.txt index f954616bb5..84648a9f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/simplified.txt @@ -1,206 +1,200 @@ -WholeStageCodegen (5) - 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] - 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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_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] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +CometNativeColumnarToRow + 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] + 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + WholeStageCodegen (2) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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_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] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometNativeScan 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + WholeStageCodegen (4) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #21 + WholeStageCodegen (3) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [d_date_sk,d_year] #12 ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt index 806525a4e2..bc532ad645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#124, d_year#125] Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#124, d_year#125] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt index 43527978d8..bfac9da68d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt index 806525a4e2..bc532ad645 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 @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#124, d_year#125] Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#124, d_year#125] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject 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 43527978d8..bfac9da68d 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 @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/explain.txt index dcf67d7592..4e2133a442 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/explain.txt @@ -6,13 +6,13 @@ TakeOrderedAndProject (43) : +- * BroadcastHashJoin Inner BuildRight (23) : :- * Project (17) : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * CometColumnarToRow (3) + : : :- CometNativeColumnarToRow (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.store (1) : : +- BroadcastExchange (15) : : +- * Filter (14) : : +- * HashAggregate (13) - : : +- * CometColumnarToRow (12) + : : +- CometNativeColumnarToRow (12) : : +- CometColumnarExchange (11) : : +- * HashAggregate (10) : : +- * Project (9) @@ -22,18 +22,18 @@ TakeOrderedAndProject (43) : : : +- Scan parquet spark_catalog.default.store_sales (4) : : +- ReusedExchange (7) : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometProject (20) : +- CometFilter (19) : +- CometNativeScan parquet spark_catalog.default.item (18) +- BroadcastExchange (40) +- * Filter (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * HashAggregate (34) - +- * CometColumnarToRow (33) + +- CometNativeColumnarToRow (33) +- CometColumnarExchange (32) +- * HashAggregate (31) +- * Project (30) @@ -55,7 +55,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) CometColumnarToRow [codegen id : 9] +(3) CometNativeColumnarToRow Input [2]: [s_store_sk#1, s_store_name#2] (4) Scan parquet spark_catalog.default.store_sales @@ -66,27 +66,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -(6) Filter [codegen id : 2] +(6) Filter [codegen id : 1] Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) (7) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#8] -(8) BroadcastHashJoin [codegen id : 2] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join type: Inner Join condition: None -(9) Project [codegen id : 2] +(9) Project [codegen id : 1] Output [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] -(10) HashAggregate [codegen id : 2] +(10) HashAggregate [codegen id : 1] Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] @@ -97,17 +97,17 @@ Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] -(14) Filter [codegen id : 3] +(14) Filter [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Condition : isnotnull(revenue#12) @@ -115,13 +115,13 @@ Condition : isnotnull(revenue#12) Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 9] +(16) BroadcastHashJoin [codegen id : 6] Left keys [1]: [s_store_sk#1] Right keys [1]: [ss_store_sk#4] Join type: Inner Join condition: None -(17) Project [codegen id : 9] +(17) Project [codegen id : 6] Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] @@ -140,20 +140,20 @@ Condition : isnotnull(i_item_sk#13) Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Arguments: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18], [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#17, 50)) AS i_brand#18] -(21) CometColumnarToRow [codegen id : 4] +(21) CometNativeColumnarToRow Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] (22) BroadcastExchange Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_item_sk#3] Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(24) Project [codegen id : 9] +(24) Project [codegen id : 6] Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] @@ -165,27 +165,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 6] +(26) ColumnarToRow [codegen id : 3] Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -(27) Filter [codegen id : 6] +(27) Filter [codegen id : 3] Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] Condition : isnotnull(ss_store_sk#20) (28) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#23] -(29) BroadcastHashJoin [codegen id : 6] +(29) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(30) Project [codegen id : 6] +(30) Project [codegen id : 3] Output [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] Input [5]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#23] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 3] Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] Keys [2]: [ss_store_sk#20, ss_item_sk#19] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] @@ -196,17 +196,17 @@ Results [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(33) CometColumnarToRow [codegen id : 7] +(33) CometNativeColumnarToRow Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -(34) HashAggregate [codegen id : 7] +(34) HashAggregate [codegen id : 4] Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] Keys [2]: [ss_store_sk#20, ss_item_sk#19] Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#21))#26] Results [2]: [ss_store_sk#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#21))#26,17,2) AS revenue#27] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 4] Input [2]: [ss_store_sk#20, revenue#27] Keys [1]: [ss_store_sk#20] Functions [1]: [partial_avg(revenue#27)] @@ -217,17 +217,17 @@ Results [3]: [ss_store_sk#20, sum#30, count#31] Input [3]: [ss_store_sk#20, sum#30, count#31] Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [3]: [ss_store_sk#20, sum#30, count#31] -(38) HashAggregate [codegen id : 8] +(38) HashAggregate [codegen id : 5] Input [3]: [ss_store_sk#20, sum#30, count#31] Keys [1]: [ss_store_sk#20] Functions [1]: [avg(revenue#27)] Aggregate Attributes [1]: [avg(revenue#27)#32] Results [2]: [ss_store_sk#20, avg(revenue#27)#32 AS ave#33] -(39) Filter [codegen id : 8] +(39) Filter [codegen id : 5] Input [2]: [ss_store_sk#20, ave#33] Condition : isnotnull(ave#33) @@ -235,13 +235,13 @@ Condition : isnotnull(ave#33) Input [2]: [ss_store_sk#20, ave#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#20] Join type: Inner Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) -(42) Project [codegen id : 9] +(42) Project [codegen id : 6] Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18, ss_store_sk#20, ave#33] @@ -253,7 +253,7 @@ Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -274,7 +274,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_mo Input [2]: [d_date_sk#8, d_month_seq#34] Arguments: [d_date_sk#8], [d_date_sk#8] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt index b1bae81b20..73e3aa7046 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt @@ -5,13 +5,13 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,28 +21,28 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -52,7 +52,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/simplified.txt index 2695c9fb89..98833264f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) + WholeStageCodegen (6) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [s_store_name,ss_store_sk,ss_item_sk,revenue] BroadcastHashJoin [s_store_sk,ss_store_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_store_name] CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [revenue] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] Project [ss_item_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -28,37 +28,33 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometNativeColumnarToRow + CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #5 - WholeStageCodegen (8) + WholeStageCodegen (5) Filter [ave] HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_store_sk] #6 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_store_sk,ss_item_sk] #7 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] Project [ss_item_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt index 6dc60931a8..0941887eb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt @@ -231,7 +231,7 @@ Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt index 1fc09bd2e3..755be00004 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt index cd1c84b6f2..bfc2fc8751 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt index 6dc60931a8..0941887eb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt @@ -231,7 +231,7 @@ Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt index 1fc09bd2e3..755be00004 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 cd1c84b6f2..bfc2fc8751 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/explain.txt index b4e8604910..fc2823f4dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (56) +- * HashAggregate (55) - +- * CometColumnarToRow (54) + +- CometNativeColumnarToRow (54) +- CometColumnarExchange (53) +- * HashAggregate (52) +- Union (51) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * Project (27) @@ -21,23 +21,23 @@ TakeOrderedAndProject (56) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.web_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) : : : +- ReusedExchange (11) : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) + : : +- CometNativeColumnarToRow (17) : : +- CometProject (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.time_dim (14) : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) + : +- CometNativeColumnarToRow (24) : +- CometProject (23) : +- CometFilter (22) : +- CometNativeScan parquet spark_catalog.default.ship_mode (21) +- * HashAggregate (50) - +- * CometColumnarToRow (49) + +- CometNativeColumnarToRow (49) +- CometColumnarExchange (48) +- * HashAggregate (47) +- * Project (46) @@ -65,10 +65,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) @@ -87,33 +87,33 @@ Condition : isnotnull(w_warehouse_sk#9) Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(w_state#14, 2)) AS w_state#16, w_country#15] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] (8) BroadcastExchange Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_warehouse_sk#3] Right keys [1]: [w_warehouse_sk#9] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] (11) ReusedExchange [Reuses operator id: 60] Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#7] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] @@ -132,20 +132,20 @@ Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= Input [2]: [t_time_sk#20, t_time#21] Arguments: [t_time_sk#20], [t_time_sk#20] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [t_time_sk#20] (18) BroadcastExchange Input [1]: [t_time_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_time_sk#1] Right keys [1]: [t_time_sk#20] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] @@ -164,24 +164,24 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_carrier#23 Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [1]: [sm_ship_mode_sk#22] (25) BroadcastExchange Input [1]: [sm_ship_mode_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_ship_mode_sk#2] Right keys [1]: [sm_ship_mode_sk#22] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] @@ -192,10 +192,10 @@ Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] @@ -210,66 +210,66 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#176), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 11] +(33) ColumnarToRow [codegen id : 3] Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] -(34) Filter [codegen id : 11] +(34) Filter [codegen id : 3] Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] Condition : ((isnotnull(cs_warehouse_sk#172) AND isnotnull(cs_sold_time_sk#170)) AND isnotnull(cs_ship_mode_sk#171)) (35) ReusedExchange [Reuses operator id: 8] Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -(36) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_warehouse_sk#172] Right keys [1]: [w_warehouse_sk#177] Join type: Inner Join condition: None -(37) Project [codegen id : 11] +(37) Project [codegen id : 3] Output [12]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] (38) ReusedExchange [Reuses operator id: 60] Output [3]: [d_date_sk#184, d_year#185, d_moy#186] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#176] Right keys [1]: [d_date_sk#184] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 3] Output [13]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [15]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] (41) ReusedExchange [Reuses operator id: 18] Output [1]: [t_time_sk#187] -(42) BroadcastHashJoin [codegen id : 11] +(42) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_time_sk#170] Right keys [1]: [t_time_sk#187] Join type: Inner Join condition: None -(43) Project [codegen id : 11] +(43) Project [codegen id : 3] Output [12]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] (44) ReusedExchange [Reuses operator id: 25] Output [1]: [sm_ship_mode_sk#188] -(45) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_ship_mode_sk#171] Right keys [1]: [sm_ship_mode_sk#188] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(46) Project [codegen id : 3] Output [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [13]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] -(47) HashAggregate [codegen id : 11] +(47) HashAggregate [codegen id : 3] Input [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] @@ -280,10 +280,10 @@ Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(49) CometColumnarToRow [codegen id : 12] +(49) CometNativeColumnarToRow Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -(50) HashAggregate [codegen id : 12] +(50) HashAggregate [codegen id : 4] Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] @@ -292,7 +292,7 @@ Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county (51) Union -(52) HashAggregate [codegen id : 13] +(52) HashAggregate [codegen id : 5] Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#146, feb_sales#147, mar_sales#148, apr_sales#149, may_sales#150, jun_sales#151, jul_sales#152, aug_sales#153, sep_sales#154, oct_sales#155, nov_sales#156, dec_sales#157, jan_net#158, feb_net#159, mar_net#160, apr_net#161, may_net#162, jun_net#163, jul_net#164, aug_net#165, sep_net#166, oct_net#167, nov_net#168, dec_net#169] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] Functions [36]: [partial_sum(jan_sales#146), partial_sum(feb_sales#147), partial_sum(mar_sales#148), partial_sum(apr_sales#149), partial_sum(may_sales#150), partial_sum(jun_sales#151), partial_sum(jul_sales#152), partial_sum(aug_sales#153), partial_sum(sep_sales#154), partial_sum(oct_sales#155), partial_sum(nov_sales#156), partial_sum(dec_sales#157), partial_sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#158), partial_sum(feb_net#159), partial_sum(mar_net#160), partial_sum(apr_net#161), partial_sum(may_net#162), partial_sum(jun_net#163), partial_sum(jul_net#164), partial_sum(aug_net#165), partial_sum(sep_net#166), partial_sum(oct_net#167), partial_sum(nov_net#168), partial_sum(dec_net#169)] @@ -303,10 +303,10 @@ Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(54) CometColumnarToRow [codegen id : 14] +(54) CometNativeColumnarToRow Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -(55) HashAggregate [codegen id : 14] +(55) HashAggregate [codegen id : 6] Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] Functions [36]: [sum(jan_sales#146), sum(feb_sales#147), sum(mar_sales#148), sum(apr_sales#149), sum(may_sales#150), sum(jun_sales#151), sum(jul_sales#152), sum(aug_sales#153), sum(sep_sales#154), sum(oct_sales#155), sum(nov_sales#156), sum(dec_sales#157), sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#158), sum(feb_net#159), sum(mar_net#160), sum(apr_net#161), sum(may_net#162), sum(jun_net#163), sum(jul_net#164), sum(aug_net#165), sum(sep_net#166), sum(oct_net#167), sum(nov_net#168), sum(dec_net#169)] @@ -321,7 +321,7 @@ Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_w Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (60) -+- * CometColumnarToRow (59) ++- CometNativeColumnarToRow (59) +- CometFilter (58) +- CometNativeScan parquet spark_catalog.default.date_dim (57) @@ -337,7 +337,7 @@ ReadSchema: struct Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) -(59) CometColumnarToRow [codegen id : 1] +(59) CometNativeColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_moy#19] (60) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt index caf49b6d52..3f3c6b8d79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,30 +21,30 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.time_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.ship_mode +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -60,21 +60,21 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.time_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/simplified.txt index 3a1f053d60..7ceb1fb345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) + WholeStageCodegen (6) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) + WholeStageCodegen (5) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] @@ -29,43 +29,35 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometNativeColumnarToRow + CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] InputAdapter ReusedExchange [d_date_sk,d_year,d_moy] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + CometNativeColumnarToRow + CometProject [t_time_sk] + CometFilter [t_time_sk,t_time] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) + CometNativeColumnarToRow + CometProject [sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] + CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) + WholeStageCodegen (3) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/explain.txt index 51db3be961..185e27d5b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/explain.txt @@ -289,7 +289,7 @@ Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -305,7 +305,7 @@ ReadSchema: struct Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_moy#19] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt index ca39e001b0..7e7f77c2c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/simplified.txt index bc6bde24b2..f3122945a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/simplified.txt @@ -21,11 +21,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt index 51db3be961..185e27d5b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt @@ -289,7 +289,7 @@ Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -305,7 +305,7 @@ ReadSchema: struct Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_moy#19] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/extended.txt index ca39e001b0..7e7f77c2c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange 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 bc6bde24b2..f3122945a6 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 @@ -21,11 +21,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/explain.txt index 1cc7d86902..0b6f505549 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/explain.txt @@ -3,13 +3,13 @@ TakeOrderedAndProject (34) +- * Filter (33) +- Window (32) +- WindowGroupLimit (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometSort (29) +- CometColumnarExchange (28) +- WindowGroupLimit (27) +- * Sort (26) +- * HashAggregate (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometColumnarExchange (23) +- * HashAggregate (22) +- * Expand (21) @@ -24,12 +24,12 @@ TakeOrderedAndProject (34) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.store (7) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometProject (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.item (14) @@ -43,23 +43,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 39] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] @@ -78,20 +78,20 @@ Condition : isnotnull(s_store_sk#11) Input [2]: [s_store_sk#11, s_store_id#12] Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#12, 16)) AS s_store_id#13] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [s_store_sk#11, s_store_id#13] (11) BroadcastExchange Input [2]: [s_store_sk#11, s_store_id#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] @@ -110,28 +110,28 @@ Condition : isnotnull(i_item_sk#14) Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#15, 50)) AS i_brand#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#16, 50)) AS i_class#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#17, 50)) AS i_category#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#18, 50)) AS i_product_name#22] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] (18) BroadcastExchange Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(21) Expand [codegen id : 4] +(21) Expand [codegen id : 1] Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#21, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 1] Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -142,17 +142,17 @@ Results [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year# Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] Arguments: hashpartitioning(i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 5] +(24) CometNativeColumnarToRow Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 2] Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36] Results [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36 AS sumsales#37] -(26) Sort [codegen id : 5] +(26) Sort [codegen id : 2] Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] Arguments: [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST], false, 0 @@ -168,7 +168,7 @@ Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, CometColumna Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] Arguments: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37], [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] (31) WindowGroupLimit @@ -179,7 +179,7 @@ Arguments: [i_category#23], [sumsales#37 DESC NULLS LAST], rank(sumsales#37), 10 Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] Arguments: [rank(sumsales#37) windowspecdefinition(i_category#23, sumsales#37 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [i_category#23], [sumsales#37 DESC NULLS LAST] -(33) Filter [codegen id : 7] +(33) Filter [codegen id : 3] Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] Condition : (rk#38 <= 100) @@ -191,7 +191,7 @@ Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometNativeScan parquet spark_catalog.default.date_dim (35) @@ -212,7 +212,7 @@ Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1200)) AND (d_mo Input [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt index e64576b0f0..da753c17c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt @@ -2,13 +2,13 @@ TakeOrderedAndProject +- Filter +- Window +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- WindowGroupLimit +- Sort +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -23,22 +23,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/simplified.txt index 7c5b24a903..31f28feccf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/simplified.txt @@ -1,57 +1,49 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) + WholeStageCodegen (3) Filter [rk] InputAdapter Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (5) - Sort [i_category,sumsales] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (2) + Sort [i_category,sumsales] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/explain.txt index 6330b854bb..b8bb87157f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -212,7 +212,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt index 7504243c7a..d03ce0e111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/simplified.txt index 9069117a5b..b4f46e52ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/simplified.txt @@ -28,12 +28,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt index 6330b854bb..b8bb87157f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt @@ -191,7 +191,7 @@ Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -212,7 +212,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/extended.txt index 7504243c7a..d03ce0e111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 9069117a5b..b4f46e52ae 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 @@ -28,12 +28,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/explain.txt index 1e2350c06a..c0de9c5394 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (41) :- * Project (37) : +- * BroadcastHashJoin Inner BuildRight (36) : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) + : : +- CometNativeColumnarToRow (29) : : +- CometColumnarExchange (28) : : +- * HashAggregate (27) : : +- * Project (26) @@ -21,21 +21,21 @@ TakeOrderedAndProject (41) : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : +- ReusedExchange (4) : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) + : : : : +- CometNativeColumnarToRow (10) : : : : +- CometProject (9) : : : : +- CometFilter (8) : : : : +- CometNativeScan parquet spark_catalog.default.store (7) : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) + : : : +- CometNativeColumnarToRow (17) : : : +- CometProject (16) : : : +- CometFilter (15) : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometFilter (22) : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) + : +- CometNativeColumnarToRow (34) : +- CometProject (33) : +- CometFilter (32) : +- CometNativeScan parquet spark_catalog.default.customer (31) @@ -50,23 +50,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#11] -(5) BroadcastHashJoin [codegen id : 5] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(6) Project [codegen id : 5] +(6) Project [codegen id : 1] Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] @@ -85,20 +85,20 @@ Condition : (s_city#13 IN (Midway,Fairview) AND isnotnull(s_store_sk#12)) Input [2]: [s_store_sk#12, s_city#13] Arguments: [s_store_sk#12], [s_store_sk#12] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#12] (11) BroadcastExchange Input [1]: [s_store_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#12] @@ -117,20 +117,20 @@ Condition : (((hd_dep_count#15 = 4) OR (hd_vehicle_count#16 = 3)) AND isnotnull( Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] Arguments: [hd_demo_sk#14], [hd_demo_sk#14] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#14] (18) BroadcastExchange Input [1]: [hd_demo_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#14] @@ -145,24 +145,24 @@ ReadSchema: struct Input [2]: [ca_address_sk#17, ca_city#18] Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_city#18)) -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [ca_address_sk#17, ca_city#18] (24) BroadcastExchange Input [2]: [ca_address_sk#17, ca_city#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#3] Right keys [1]: [ca_address_sk#17] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(26) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#17, ca_city#18] -(27) HashAggregate [codegen id : 5] +(27) HashAggregate [codegen id : 1] Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] @@ -173,10 +173,10 @@ Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, su Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometColumnarToRow [codegen id : 8] +(29) CometNativeColumnarToRow Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -(30) HashAggregate [codegen id : 8] +(30) HashAggregate [codegen id : 2] Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] @@ -198,33 +198,33 @@ Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] Arguments: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37], [c_customer_sk#32, c_current_addr_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#34, 20)) AS c_first_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#35, 30)) AS c_last_name#37] -(34) CometColumnarToRow [codegen id : 6] +(34) CometNativeColumnarToRow Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] (35) BroadcastExchange Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 2] Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37] Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] (38) ReusedExchange [Reuses operator id: 24] Output [2]: [ca_address_sk#38, ca_city#39] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_current_addr_sk#33] Right keys [1]: [ca_address_sk#38] Join type: Inner Join condition: NOT (ca_city#39 = bought_city#28) -(40) Project [codegen id : 8] +(40) Project [codegen id : 2] Output [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_city#39] @@ -236,7 +236,7 @@ Arguments: 100, [c_last_name#37 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FI Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometNativeScan parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((((isnotnull(d_dom#41) AND (d_dom#41 >= 1)) AND (d_dom#41 <= 2)) AN Input [3]: [d_date_sk#11, d_year#40, d_dom#41] Arguments: [d_date_sk#11], [d_date_sk#11] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt index c122bf3803..cebcd85144 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -20,36 +20,36 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/simplified.txt index 05b883b1e6..5a807ddd72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -24,44 +24,34 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometNativeColumnarToRow + CometFilter [ca_address_sk,ca_city] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt index 02afc31b71..822475956e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt @@ -225,7 +225,7 @@ Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt index fdddd82de0..ee6be5a221 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt index 02afc31b71..822475956e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt @@ -225,7 +225,7 @@ Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fdddd82de0..ee6be5a221 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/explain.txt index cd1eacdb81..e77c6d46bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (44) +- * HashAggregate (43) - +- * CometColumnarToRow (42) + +- CometNativeColumnarToRow (42) +- CometColumnarExchange (41) +- * HashAggregate (40) +- * Project (39) @@ -12,7 +12,7 @@ TakeOrderedAndProject (44) : : +- * BroadcastHashJoin LeftAnti BuildRight (24) : : :- * BroadcastHashJoin LeftAnti BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -34,12 +34,12 @@ TakeOrderedAndProject (44) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (30) - : +- * CometColumnarToRow (29) + : +- CometNativeColumnarToRow (29) : +- CometProject (28) : +- CometFilter (27) : +- CometNativeScan parquet spark_catalog.default.customer_address (26) +- BroadcastExchange (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.customer_demographics (33) @@ -56,7 +56,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] (6) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] @@ -86,7 +86,7 @@ Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi @@ -99,19 +99,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] (13) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#10] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#8] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] @@ -119,7 +119,7 @@ Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] Input [1]: [ws_bill_customer_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#8] Join type: LeftAnti @@ -132,19 +132,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#6)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] (20) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#13] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#11] Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] @@ -152,13 +152,13 @@ Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] Input [1]: [cs_ship_customer_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#11] Join type: LeftAnti Join condition: None -(25) Project [codegen id : 9] +(25) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -177,20 +177,20 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#15, Input [2]: [ca_address_sk#14, ca_state#15] Arguments: [ca_address_sk#14], [ca_address_sk#14] -(29) CometColumnarToRow [codegen id : 7] +(29) CometNativeColumnarToRow Input [1]: [ca_address_sk#14] (30) BroadcastExchange Input [1]: [ca_address_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 9] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#14] Join type: Inner Join condition: None -(32) Project [codegen id : 9] +(32) Project [codegen id : 4] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#14] @@ -209,24 +209,24 @@ Condition : isnotnull(cd_demo_sk#16) Input [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] Arguments: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25], [cd_demo_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#17, 1)) AS cd_gender#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#18, 1)) AS cd_marital_status#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#19, 20)) AS cd_education_status#24, cd_purchase_estimate#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#21, 10)) AS cd_credit_rating#25] -(36) CometColumnarToRow [codegen id : 8] +(36) CometNativeColumnarToRow Input [6]: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] (37) BroadcastExchange Input [6]: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#16] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(39) Project [codegen id : 4] Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 4] Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Functions [1]: [partial_count(1)] @@ -237,10 +237,10 @@ Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_pur Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometColumnarToRow [codegen id : 10] +(42) CometNativeColumnarToRow Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] -(43) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 5] Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] Functions [1]: [count(1)] @@ -255,7 +255,7 @@ Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FI Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometNativeScan parquet spark_catalog.default.date_dim (45) @@ -276,7 +276,7 @@ Condition : (((((isnotnull(d_year#32) AND isnotnull(d_moy#33)) AND (d_year#32 = Input [3]: [d_date_sk#7, d_year#32, d_moy#33] Arguments: [d_date_sk#7], [d_date_sk#7] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt index 821b101f54..f48c67d6e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -21,12 +21,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,7 +37,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -48,17 +48,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/simplified.txt index f8868e38be..e5b4d44c3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -14,13 +14,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -28,17 +28,15 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -49,7 +47,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -60,17 +58,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/explain.txt index 96075ed3ec..b28c8e5488 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/explain.txt @@ -36,12 +36,12 @@ TakeOrderedAndProject (46) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) + : +- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometFilter (29) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) +- BroadcastExchange (39) - +- * CometColumnarToRow (38) + +- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) @@ -102,7 +102,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -131,7 +131,7 @@ Input [1]: [ws_bill_customer_sk#10] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#10] Join type: LeftAnti @@ -163,13 +163,13 @@ Input [1]: [cs_ship_customer_sk#14] Input [1]: [cs_ship_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#14] Join type: LeftAnti Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -188,20 +188,20 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#19, Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [ca_address_sk#18] (32) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 5] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(34) Project [codegen id : 5] +(34) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] @@ -220,24 +220,24 @@ Condition : isnotnull(cd_demo_sk#20) Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Arguments: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29], [cd_demo_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#21, 1)) AS cd_gender#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#22, 1)) AS cd_marital_status#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#23, 20)) AS cd_education_status#28, cd_purchase_estimate#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#25, 10)) AS cd_credit_rating#29] -(38) CometColumnarToRow [codegen id : 4] +(38) CometNativeColumnarToRow Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] (39) BroadcastExchange Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 5] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(41) Project [codegen id : 5] +(41) Project [codegen id : 3] Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -(42) HashAggregate [codegen id : 5] +(42) HashAggregate [codegen id : 3] Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Functions [1]: [partial_count(1)] @@ -248,10 +248,10 @@ Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_pur Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) CometColumnarToRow [codegen id : 6] +(44) CometColumnarToRow [codegen id : 4] Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -(45) HashAggregate [codegen id : 6] +(45) HashAggregate [codegen id : 4] Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Functions [1]: [count(1)] @@ -266,7 +266,7 @@ Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (51) -+- * CometColumnarToRow (50) ++- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) @@ -287,7 +287,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(50) CometColumnarToRow [codegen id : 1] +(50) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt index b4fbb455ef..5a472217df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -49,12 +49,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/simplified.txt index 68bf32c40b..fd8db7bed2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -24,12 +24,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -56,17 +54,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt index 96075ed3ec..b28c8e5488 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt @@ -36,12 +36,12 @@ TakeOrderedAndProject (46) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) + : +- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometFilter (29) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) +- BroadcastExchange (39) - +- * CometColumnarToRow (38) + +- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) @@ -102,7 +102,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -131,7 +131,7 @@ Input [1]: [ws_bill_customer_sk#10] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#10] Join type: LeftAnti @@ -163,13 +163,13 @@ Input [1]: [cs_ship_customer_sk#14] Input [1]: [cs_ship_customer_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#14] Join type: LeftAnti Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -188,20 +188,20 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#19, Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [ca_address_sk#18] (32) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 5] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(34) Project [codegen id : 5] +(34) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] @@ -220,24 +220,24 @@ Condition : isnotnull(cd_demo_sk#20) Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] Arguments: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29], [cd_demo_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#21, 1)) AS cd_gender#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#22, 1)) AS cd_marital_status#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#23, 20)) AS cd_education_status#28, cd_purchase_estimate#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#25, 10)) AS cd_credit_rating#29] -(38) CometColumnarToRow [codegen id : 4] +(38) CometNativeColumnarToRow Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] (39) BroadcastExchange Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 5] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(41) Project [codegen id : 5] +(41) Project [codegen id : 3] Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -(42) HashAggregate [codegen id : 5] +(42) HashAggregate [codegen id : 3] Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Functions [1]: [partial_count(1)] @@ -248,10 +248,10 @@ Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_pur Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) CometColumnarToRow [codegen id : 6] +(44) CometColumnarToRow [codegen id : 4] Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -(45) HashAggregate [codegen id : 6] +(45) HashAggregate [codegen id : 4] Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] Functions [1]: [count(1)] @@ -266,7 +266,7 @@ Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (51) -+- * CometColumnarToRow (50) ++- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) @@ -287,7 +287,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(50) CometColumnarToRow [codegen id : 1] +(50) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/extended.txt index b4fbb455ef..5a472217df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -49,12 +49,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt index 68bf32c40b..fd8db7bed2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -24,12 +24,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -56,17 +54,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/explain.txt index 627eb9e23d..c6f20968b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * HashAggregate (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * Project (27) @@ -16,18 +16,18 @@ TakeOrderedAndProject (32) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : +- ReusedExchange (11) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.item (14) +- BroadcastExchange (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometNativeScan parquet spark_catalog.default.promotion (21) @@ -41,10 +41,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) @@ -63,33 +63,33 @@ Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) A Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -108,20 +108,20 @@ Condition : isnotnull(i_item_sk#15) Input [2]: [i_item_sk#15, i_item_id#16] Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#16, 16)) AS i_item_id#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [i_item_sk#15, i_item_id#17] (18) BroadcastExchange Input [2]: [i_item_sk#15, i_item_id#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#15, i_item_id#17] @@ -140,24 +140,24 @@ Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_e Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Arguments: [p_promo_sk#18], [p_promo_sk#18] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [1]: [p_promo_sk#18] (25) BroadcastExchange Input [1]: [p_promo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_promo_sk#3] Right keys [1]: [p_promo_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17, p_promo_sk#18] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] @@ -168,10 +168,10 @@ Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34 Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Keys [1]: [i_item_id#17] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] @@ -186,7 +186,7 @@ Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#45] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt index f40fdbe4e3..b692dfa9be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -16,27 +16,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/simplified.txt index bb670b4a73..d7988608ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] BroadcastHashJoin [ss_promo_sk,p_promo_sk] @@ -20,35 +20,27 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometNativeColumnarToRow + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/explain.txt index b1e3b154ac..0ce5aa49f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt index 655f651a41..3aa5ad7551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/simplified.txt index a15bcd5c0d..707bfe627e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/simplified.txt @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt index b1e3b154ac..0ce5aa49f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/extended.txt index 655f651a41..3aa5ad7551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a15bcd5c0d..707bfe627e 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 @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/explain.txt index 071f670c73..23852fc91c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- CometColumnarExchange (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Expand (38) @@ -21,7 +21,7 @@ TakeOrderedAndProject (48) +- BroadcastExchange (35) +- * Project (34) +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (9) + :- CometNativeColumnarToRow (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.store (7) +- BroadcastExchange (32) @@ -31,7 +31,7 @@ TakeOrderedAndProject (48) +- WindowGroupLimit (28) +- * Sort (27) +- * HashAggregate (26) - +- * CometColumnarToRow (25) + +- CometNativeColumnarToRow (25) +- CometColumnarExchange (24) +- * HashAggregate (23) +- * Project (22) @@ -42,7 +42,7 @@ TakeOrderedAndProject (48) : : +- * ColumnarToRow (11) : : +- Scan parquet spark_catalog.default.store_sales (10) : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometProject (15) : +- CometFilter (14) : +- CometNativeScan parquet spark_catalog.default.store (13) @@ -57,23 +57,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 5] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -88,7 +88,7 @@ ReadSchema: struct Input [3]: [s_store_sk#6, s_county#7, s_state#8] Condition : isnotnull(s_store_sk#6) -(9) CometColumnarToRow [codegen id : 7] +(9) CometNativeColumnarToRow Input [3]: [s_store_sk#6, s_county#7, s_state#8] (10) Scan parquet spark_catalog.default.store_sales @@ -99,10 +99,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Condition : isnotnull(ss_store_sk#9) @@ -121,37 +121,37 @@ Condition : isnotnull(s_store_sk#12) Input [2]: [s_store_sk#12, s_state#13] Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#14] -(16) CometColumnarToRow [codegen id : 2] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#12, s_state#14] (17) BroadcastExchange Input [2]: [s_store_sk#12, s_state#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#9] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] (20) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 4] +(21) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#11] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 4] +(22) Project [codegen id : 1] Output [2]: [ss_net_profit#10, s_state#14] Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] -(23) HashAggregate [codegen id : 4] +(23) HashAggregate [codegen id : 1] Input [2]: [ss_net_profit#10, s_state#14] Keys [1]: [s_state#14] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] @@ -162,17 +162,17 @@ Results [2]: [s_state#14, sum#17] Input [2]: [s_state#14, sum#17] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(25) CometColumnarToRow [codegen id : 5] +(25) CometNativeColumnarToRow Input [2]: [s_state#14, sum#17] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 2] Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(27) Sort [codegen id : 5] +(27) Sort [codegen id : 2] Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 @@ -184,11 +184,11 @@ Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] -(30) Filter [codegen id : 6] +(30) Filter [codegen id : 3] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Condition : (ranking#20 <= 5) -(31) Project [codegen id : 6] +(31) Project [codegen id : 3] Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] @@ -196,13 +196,13 @@ Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 7] +(33) BroadcastHashJoin [codegen id : 4] Left keys [1]: [static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#8, 2))] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(34) Project [codegen id : 7] +(34) Project [codegen id : 4] Output [3]: [s_store_sk#6, s_county#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#8, 2)) AS s_state#21] Input [3]: [s_store_sk#6, s_county#7, s_state#8] @@ -210,21 +210,21 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] Input [3]: [s_store_sk#6, s_county#7, s_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 5] Output [3]: [ss_net_profit#2, s_state#21, s_county#7] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] -(38) Expand [codegen id : 8] +(38) Expand [codegen id : 5] Input [3]: [ss_net_profit#2, s_state#21, s_county#7] Arguments: [[ss_net_profit#2, s_state#21, s_county#7, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 5] Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] @@ -235,10 +235,10 @@ Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 6] Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] @@ -253,14 +253,14 @@ Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] -(45) CometColumnarToRow [codegen id : 10] +(45) CometNativeColumnarToRow Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] (46) Window Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] -(47) Project [codegen id : 11] +(47) Project [codegen id : 7] Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] @@ -272,7 +272,7 @@ Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -293,7 +293,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#34] Arguments: [d_date_sk#5], [d_date_sk#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt index 4832534e6e..f222353c17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -18,19 +18,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange @@ -40,7 +40,7 @@ TakeOrderedAndProject +- WindowGroupLimit +- Sort +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -52,12 +52,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/simplified.txt index f61238590a..21c9c4dbc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/simplified.txt @@ -1,80 +1,74 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) + WholeStageCodegen (7) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (9) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (6) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + InputAdapter + CometNativeColumnarToRow + CometFilter [s_store_sk,s_county,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (2) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state] #6 + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt index 44f6ce791c..4a62d4a581 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject (49) +- BroadcastExchange (36) +- * Project (35) +- * BroadcastHashJoin LeftSemi BuildRight (34) - :- * CometColumnarToRow (12) + :- CometNativeColumnarToRow (12) : +- CometFilter (11) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) +- BroadcastExchange (33) @@ -104,7 +104,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -276,7 +276,7 @@ Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -297,7 +297,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt index 45a2c7a669..0239787721 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -29,7 +29,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt index 80f0cfc8c2..d3262b3181 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -41,8 +39,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt index 44f6ce791c..4a62d4a581 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject (49) +- BroadcastExchange (36) +- * Project (35) +- * BroadcastHashJoin LeftSemi BuildRight (34) - :- * CometColumnarToRow (12) + :- CometNativeColumnarToRow (12) : +- CometFilter (11) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) +- BroadcastExchange (33) @@ -104,7 +104,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -276,7 +276,7 @@ Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -297,7 +297,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt index 45a2c7a669..0239787721 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -29,7 +29,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange 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 80f0cfc8c2..d3262b3181 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -41,8 +39,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/explain.txt index d0c100dc27..a3a46b1544 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (40) +CometNativeColumnarToRow (40) +- CometSort (39) +- CometColumnarExchange (38) +- * HashAggregate (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometColumnarExchange (35) +- * HashAggregate (34) +- * Project (33) @@ -11,7 +11,7 @@ :- * Project (26) : +- * BroadcastHashJoin Inner BuildLeft (25) : :- BroadcastExchange (5) - : : +- * CometColumnarToRow (4) + : : +- CometNativeColumnarToRow (4) : : +- CometProject (3) : : +- CometFilter (2) : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -35,7 +35,7 @@ : : +- Scan parquet spark_catalog.default.store_sales (18) : +- ReusedExchange (21) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.time_dim (27) @@ -56,7 +56,7 @@ Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull( Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#3, 50)) AS i_brand#5] -(4) CometColumnarToRow [codegen id : 1] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] (5) BroadcastExchange @@ -71,23 +71,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 3] +(7) ColumnarToRow [codegen id : 1] Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -(8) Filter [codegen id : 3] +(8) Filter [codegen id : 1] Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) (9) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#11] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 1] Output [3]: [ws_ext_sales_price#8 AS ext_price#12, ws_item_sk#7 AS sold_item_sk#13, ws_sold_time_sk#6 AS time_sk#14] Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] @@ -99,23 +99,23 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 5] +(13) ColumnarToRow [codegen id : 2] Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -(14) Filter [codegen id : 5] +(14) Filter [codegen id : 2] Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] Condition : (isnotnull(cs_item_sk#16) AND isnotnull(cs_sold_time_sk#15)) (15) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#19] -(16) BroadcastHashJoin [codegen id : 5] +(16) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(17) Project [codegen id : 5] +(17) Project [codegen id : 2] Output [3]: [cs_ext_sales_price#17 AS ext_price#20, cs_item_sk#16 AS sold_item_sk#21, cs_sold_time_sk#15 AS time_sk#22] Input [5]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#19] @@ -127,35 +127,35 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 3] Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(20) Filter [codegen id : 7] +(20) Filter [codegen id : 3] Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) (21) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#27] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 3] Output [3]: [ss_ext_sales_price#25 AS ext_price#28, ss_item_sk#24 AS sold_item_sk#29, ss_sold_time_sk#23 AS time_sk#30] Input [5]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] (24) Union -(25) BroadcastHashJoin [codegen id : 9] +(25) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] Right keys [1]: [sold_item_sk#13] Join type: Inner Join condition: None -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [4]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#12, sold_item_sk#13, time_sk#14] @@ -174,24 +174,24 @@ Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(t_meal_time Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] Arguments: [t_time_sk#31, t_hour#32, t_minute#33], [t_time_sk#31, t_hour#32, t_minute#33] -(30) CometColumnarToRow [codegen id : 8] +(30) CometNativeColumnarToRow Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] (31) BroadcastExchange Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [time_sk#14] Right keys [1]: [t_time_sk#31] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] Input [7]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14, t_time_sk#31, t_hour#32, t_minute#33] -(34) HashAggregate [codegen id : 9] +(34) HashAggregate [codegen id : 4] Input [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] Functions [1]: [partial_sum(UnscaledValue(ext_price#12))] @@ -202,10 +202,10 @@ Results [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(36) CometColumnarToRow [codegen id : 10] +(36) CometNativeColumnarToRow Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] -(37) HashAggregate [codegen id : 10] +(37) HashAggregate [codegen id : 5] Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] Functions [1]: [sum(UnscaledValue(ext_price#12))] @@ -220,14 +220,14 @@ Arguments: rangepartitioning(ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] Arguments: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40], [ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 11] +(40) CometNativeColumnarToRow Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.date_dim (41) @@ -248,7 +248,7 @@ Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 11 Input [3]: [d_date_sk#11, d_year#41, d_moy#42] Arguments: [d_date_sk#11], [d_date_sk#11] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt index 427d3518dc..23844f41f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ CometColumnarToRow :- Project : +- BroadcastHashJoin : :- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -22,12 +22,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,12 +49,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/simplified.txt index 838a3e6604..9fdaac52c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/simplified.txt @@ -1,71 +1,63 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 - WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + CometSort [brand_id,brand,t_hour,t_minute,ext_price] + CometColumnarExchange [ext_price,brand_id] #1 + WholeStageCodegen (5) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + WholeStageCodegen (4) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] + Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] + BroadcastHashJoin [time_sk,t_time_sk] + Project [i_brand_id,i_brand,ext_price,time_sk] + BroadcastHashJoin [i_item_sk,sold_item_sk] + InputAdapter + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + InputAdapter + Union + WholeStageCodegen (1) + Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_time_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (2) + Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_time_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (3) + Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_time_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + Scan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt index 00ff66d3eb..92ea265885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt @@ -216,7 +216,7 @@ Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11 Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt index 25ea15c536..12dc136fee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt index 63e7c353ac..0c925fadcb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt index 00ff66d3eb..92ea265885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt @@ -216,7 +216,7 @@ Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11 Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt index 25ea15c536..12dc136fee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 63e7c353ac..0c925fadcb 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/explain.txt index 34b69fc8d2..cb0b067f01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (70) +CometNativeColumnarToRow (70) +- CometTakeOrderedAndProject (69) +- CometHashAggregate (68) +- CometExchange (67) @@ -30,38 +30,38 @@ : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) + : : : : : : : : : +- CometNativeColumnarToRow (6) : : : : : : : : : +- CometFilter (5) : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) + : : : : : : : : +- CometNativeColumnarToRow (12) : : : : : : : : +- CometFilter (11) : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometNativeColumnarToRow (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) + : : : : : : +- CometNativeColumnarToRow (25) : : : : : : +- CometProject (24) : : : : : : +- CometFilter (23) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) + : : : : : +- CometNativeColumnarToRow (32) : : : : : +- CometProject (31) : : : : : +- CometFilter (30) : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) : : : : +- ReusedExchange (36) : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) + : : : +- CometNativeColumnarToRow (41) : : : +- CometFilter (40) : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) + : : +- CometNativeColumnarToRow (47) : : +- CometFilter (46) : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometFilter (52) : +- CometNativeScan parquet spark_catalog.default.promotion (51) +- CometSort (63) @@ -79,10 +79,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 10] +(2) ColumnarToRow [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -(3) Filter [codegen id : 10] +(3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) @@ -98,20 +98,20 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] (13) BroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 10] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#11] Right keys [1]: [w_warehouse_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 10] +(15) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] @@ -154,20 +154,20 @@ ReadSchema: struct Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) CometColumnarToRow [codegen id : 3] +(18) CometNativeColumnarToRow Input [2]: [i_item_sk#16, i_item_desc#17] (19) BroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 10] +(20) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#4] Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(21) Project [codegen id : 10] +(21) Project [codegen id : 1] Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] @@ -186,20 +186,20 @@ Condition : ((isnotnull(cd_marital_status#19) AND (static_invoke(CharVarcharCode Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) CometColumnarToRow [codegen id : 4] +(25) CometNativeColumnarToRow Input [1]: [cd_demo_sk#18] (26) BroadcastExchange Input [1]: [cd_demo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 10] +(27) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(28) Project [codegen id : 10] +(28) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] @@ -218,33 +218,33 @@ Condition : ((isnotnull(hd_buy_potential#21) AND (static_invoke(CharVarcharCodeg Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) CometColumnarToRow [codegen id : 5] +(32) CometNativeColumnarToRow Input [1]: [hd_demo_sk#20] (33) BroadcastExchange Input [1]: [hd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_hdemo_sk#3] Right keys [1]: [hd_demo_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 10] +(35) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] (36) ReusedExchange [Reuses operator id: 75] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(37) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#8] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(38) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] @@ -259,20 +259,20 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_week_seq#26] Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) -(41) CometColumnarToRow [codegen id : 7] +(41) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_week_seq#26] (42) BroadcastExchange Input [2]: [d_date_sk#25, d_week_seq#26] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 10] +(43) BroadcastHashJoin [codegen id : 1] Left keys [2]: [d_week_seq#24, inv_date_sk#13] Right keys [2]: [d_week_seq#26, d_date_sk#25] Join type: Inner Join condition: None -(44) Project [codegen id : 10] +(44) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] @@ -287,20 +287,20 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_date#28] Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) -(47) CometColumnarToRow [codegen id : 8] +(47) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_date#28] (48) BroadcastExchange Input [2]: [d_date_sk#27, d_date#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: (d_date#28 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(50) Project [codegen id : 1] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] @@ -315,20 +315,20 @@ ReadSchema: struct Input [1]: [p_promo_sk#29] Condition : isnotnull(p_promo_sk#29) -(53) CometColumnarToRow [codegen id : 9] +(53) CometNativeColumnarToRow Input [1]: [p_promo_sk#29] (54) BroadcastExchange Input [1]: [p_promo_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(55) BroadcastHashJoin [codegen id : 10] +(55) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_promo_sk#5] Right keys [1]: [p_promo_sk#29] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(56) Project [codegen id : 1] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] @@ -390,14 +390,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -(70) CometColumnarToRow [codegen id : 11] +(70) CometNativeColumnarToRow Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (75) -+- * CometColumnarToRow (74) ++- CometNativeColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -418,7 +418,7 @@ Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) CometColumnarToRow [codegen id : 1] +(74) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt index 081972705a..62c2b17b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -30,47 +30,47 @@ CometColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/simplified.txt index 025217a607..edcf2b00f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/simplified.txt @@ -1,107 +1,87 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - 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] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + 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] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + BroadcastExchange #6 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + BroadcastExchange #9 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + BroadcastExchange #10 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + CometNativeColumnarToRow + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/explain.txt index 7287cba3be..d3d5646fb3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/simplified.txt index 678f8c67df..549469c03b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt index 7287cba3be..d3d5646fb3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 678f8c67df..549469c03b 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 @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/explain.txt index 9cb3536ab0..ec3b452693 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometSort (34) +- CometColumnarExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) :- * Filter (25) : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -20,17 +20,17 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.customer (26) @@ -44,23 +44,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] @@ -79,20 +79,20 @@ Condition : (s_county#9 IN (Williamson County,Franklin Parish,Bronx County,Orang Input [2]: [s_store_sk#8, s_county#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (11) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] @@ -111,24 +111,24 @@ Condition : ((((isnotnull(hd_vehicle_count#13) AND ((static_invoke(CharVarcharCo Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#10] (18) BroadcastExchange Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] @@ -139,17 +139,17 @@ Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) @@ -168,20 +168,20 @@ Condition : isnotnull(c_customer_sk#18) Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] (30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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] @@ -193,14 +193,14 @@ Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, Co 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: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (40) -+- * CometColumnarToRow (39) ++- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometNativeScan parquet spark_catalog.default.date_dim (36) @@ -221,7 +221,7 @@ Condition : ((((isnotnull(d_dom#28) AND (d_dom#28 >= 1)) AND (d_dom#28 <= 2)) AN Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(39) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (40) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt index 9e4596c12c..40230582bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- BroadcastHashJoin :- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,27 +19,27 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/simplified.txt index 4d2a6a74ab..ef3860150b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/simplified.txt @@ -1,59 +1,49 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] +CometNativeColumnarToRow + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [cnt] #1 + WholeStageCodegen (2) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/explain.txt index 3bb13dcb30..e6fe189754 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/simplified.txt index 227dd00f2b..133182ebde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 3bb13dcb30..e6fe189754 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 @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 227dd00f2b..133182ebde 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/explain.txt index 174c3d72a2..00542a51e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/explain.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject (79) : :- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -25,14 +25,14 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -44,14 +44,14 @@ TakeOrderedAndProject (79) : +- BroadcastExchange (56) : +- * Filter (55) : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (47) : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) + : : :- CometNativeColumnarToRow (41) : : : +- CometProject (40) : : : +- CometFilter (39) : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -62,14 +62,14 @@ TakeOrderedAndProject (79) : +- ReusedExchange (48) +- BroadcastExchange (76) +- * HashAggregate (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometColumnarExchange (73) +- * HashAggregate (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) :- * Project (68) : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) + : :- CometNativeColumnarToRow (62) : : +- CometProject (61) : : +- CometFilter (60) : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -95,7 +95,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharC Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#7] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] (5) Scan parquet spark_catalog.default.store_sales @@ -117,30 +117,30 @@ Condition : isnotnull(ss_customer_sk#8) Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] (11) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#12, d_year#13] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] @@ -151,17 +151,17 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) @@ -180,7 +180,7 @@ Condition : (isnotnull(c_customer_sk#19) AND isnotnull(static_invoke(CharVarchar Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Arguments: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25], [c_customer_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#20, 16)) AS c_customer_id#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#21, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#22, 30)) AS c_last_name#25] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [4]: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25] (23) Scan parquet spark_catalog.default.store_sales @@ -191,10 +191,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] Condition : isnotnull(ss_customer_sk#26) @@ -202,30 +202,30 @@ Condition : isnotnull(ss_customer_sk#26) Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#19] Right keys [1]: [ss_customer_sk#26] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, ss_sold_date_sk#28] Input [7]: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25, ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] (29) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#30, d_year#31] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#28] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, d_year#31] Input [7]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, d_year#31] Keys [4]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#27))] @@ -236,10 +236,10 @@ Results [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum# Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] Arguments: hashpartitioning(c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] Keys [4]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31] Functions [1]: [sum(UnscaledValue(ss_net_paid#27))] @@ -250,7 +250,7 @@ Results [4]: [c_customer_id#23 AS customer_id#34, c_first_name#24 AS customer_fi Input [4]: [customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#34] Join type: Inner @@ -271,7 +271,7 @@ Condition : (isnotnull(c_customer_sk#38) AND isnotnull(static_invoke(CharVarchar Input [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] Arguments: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44], [c_customer_sk#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#39, 16)) AS c_customer_id#42, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#40, 20)) AS c_first_name#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#41, 30)) AS c_last_name#44] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [4]: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44] (42) Scan parquet spark_catalog.default.web_sales @@ -282,10 +282,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] Condition : isnotnull(ws_bill_customer_sk#45) @@ -293,30 +293,30 @@ Condition : isnotnull(ws_bill_customer_sk#45) Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#38] Right keys [1]: [ws_bill_customer_sk#45] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, ws_sold_date_sk#47] Input [7]: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44, ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] (48) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#48, d_year#49] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, d_year#49] Input [7]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, ws_sold_date_sk#47, d_date_sk#48, d_year#49] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, d_year#49] Keys [4]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#46))] @@ -327,17 +327,17 @@ Results [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum# Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] Keys [4]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49] Functions [1]: [sum(UnscaledValue(ws_net_paid#46))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#46))#52] Results [2]: [c_customer_id#42 AS customer_id#53, MakeDecimal(sum(UnscaledValue(ws_net_paid#46))#52,17,2) AS year_total#54] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#53, year_total#54] Condition : (isnotnull(year_total#54) AND (year_total#54 > 0.00)) @@ -345,13 +345,13 @@ Condition : (isnotnull(year_total#54) AND (year_total#54 > 0.00)) Input [2]: [customer_id#53, year_total#54] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#53] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(58) Project [codegen id : 12] Output [7]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54] Input [8]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, customer_id#53, year_total#54] @@ -370,7 +370,7 @@ Condition : (isnotnull(c_customer_sk#55) AND isnotnull(static_invoke(CharVarchar Input [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] Arguments: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61], [c_customer_sk#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#56, 16)) AS c_customer_id#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#57, 20)) AS c_first_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#58, 30)) AS c_last_name#61] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [4]: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61] (63) Scan parquet spark_catalog.default.web_sales @@ -381,10 +381,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] Condition : isnotnull(ws_bill_customer_sk#62) @@ -392,30 +392,30 @@ Condition : isnotnull(ws_bill_customer_sk#62) Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#55] Right keys [1]: [ws_bill_customer_sk#62] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64] Input [7]: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61, ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] (69) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#65, d_year#66] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#64] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#66] Input [7]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#66] Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#63))] @@ -426,10 +426,10 @@ Results [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum# Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66] Functions [1]: [sum(UnscaledValue(ws_net_paid#63))] @@ -440,13 +440,13 @@ Results [2]: [c_customer_id#59 AS customer_id#69, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#69, year_total#70] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#69] Join type: Inner Join condition: (CASE WHEN (year_total#54 > 0.00) THEN (year_total#70 / year_total#54) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#37 / year_total#18) END) -(78) Project [codegen id : 16] +(78) Project [codegen id : 12] Output [3]: [customer_id#34, customer_first_name#35, customer_last_name#36] Input [9]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54, customer_id#69, year_total#70] @@ -458,7 +458,7 @@ Arguments: 100, [customer_id#34 ASC NULLS FIRST, customer_id#34 ASC NULLS FIRST, Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometFilter (81) +- CometNativeScan parquet spark_catalog.default.date_dim (80) @@ -474,7 +474,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (83) BroadcastExchange @@ -483,7 +483,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (87) -+- * CometColumnarToRow (86) ++- CometNativeColumnarToRow (86) +- CometFilter (85) +- CometNativeScan parquet spark_catalog.default.date_dim (84) @@ -499,7 +499,7 @@ ReadSchema: struct Input [2]: [d_date_sk#30, d_year#31] Condition : (((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND d_year#31 IN (2001,2002)) AND isnotnull(d_date_sk#30)) -(86) CometColumnarToRow [codegen id : 1] +(86) CometNativeColumnarToRow Input [2]: [d_date_sk#30, d_year#31] (87) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt index 0d59f7b833..5d218998be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -23,23 +23,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -49,24 +49,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -76,19 +76,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -98,7 +98,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/simplified.txt index f488e18c7c..01e22b7b4a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -7,17 +7,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] @@ -30,69 +30,65 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -102,25 +98,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt index ce451747c0..66337ac33a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#28, d_year#29] Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_year#29] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt index 72e39422e0..9ec698fdca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt index ce451747c0..66337ac33a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#28, d_year#29] Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_year#29] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 72e39422e0..9ec698fdca 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 @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/explain.txt index 87315ef48b..8ab8d09e45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (127) +CometNativeColumnarToRow (127) +- CometTakeOrderedAndProject (126) +- CometProject (125) +- CometSortMergeJoin (124) @@ -25,7 +25,7 @@ : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -136,10 +136,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Condition : isnotnull(cs_item_sk#1) @@ -158,33 +158,33 @@ Condition : ((((((isnotnull(i_category#11) AND (static_invoke(CharVarcharCodegen Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (8) BroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (11) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#13, d_year#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] @@ -236,36 +236,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(24) ColumnarToRow [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) (26) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#22] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] (29) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#32, d_year#33] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] @@ -317,36 +317,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 9] +(42) ColumnarToRow [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -(43) Filter [codegen id : 9] +(43) Filter [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Condition : isnotnull(ws_item_sk#41) (44) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -(45) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_item_sk#41] Right keys [1]: [i_item_sk#46] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(46) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] (47) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#51, d_year#52] -(48) BroadcastHashJoin [codegen id : 9] +(48) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#45] Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(49) Project [codegen id : 9] +(49) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] @@ -443,36 +443,36 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 12] +(70) ColumnarToRow [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -(71) Filter [codegen id : 12] +(71) Filter [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] Condition : isnotnull(cs_item_sk#64) (72) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -(73) BroadcastHashJoin [codegen id : 12] +(73) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_item_sk#64] Right keys [1]: [i_item_sk#70] Join type: Inner Join condition: None -(74) Project [codegen id : 12] +(74) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] (75) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#75, d_year#76] -(76) BroadcastHashJoin [codegen id : 12] +(76) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#68] Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(77) Project [codegen id : 12] +(77) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] @@ -508,36 +508,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 15] +(85) ColumnarToRow [codegen id : 5] Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -(86) Filter [codegen id : 15] +(86) Filter [codegen id : 5] Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Condition : isnotnull(ss_item_sk#83) (87) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] -(88) BroadcastHashJoin [codegen id : 15] +(88) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#83] Right keys [1]: [i_item_sk#88] Join type: Inner Join condition: None -(89) Project [codegen id : 15] +(89) Project [codegen id : 5] Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] (90) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#93, d_year#94] -(91) BroadcastHashJoin [codegen id : 15] +(91) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#87] Right keys [1]: [d_date_sk#93] Join type: Inner Join condition: None -(92) Project [codegen id : 15] +(92) Project [codegen id : 5] Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_date_sk#93, d_year#94] @@ -573,36 +573,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 18] +(100) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -(101) Filter [codegen id : 18] +(101) Filter [codegen id : 6] Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] Condition : isnotnull(ws_item_sk#101) (102) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(103) BroadcastHashJoin [codegen id : 18] +(103) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#101] Right keys [1]: [i_item_sk#106] Join type: Inner Join condition: None -(104) Project [codegen id : 18] +(104) Project [codegen id : 6] Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] Input [10]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] (105) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#111, d_year#112] -(106) BroadcastHashJoin [codegen id : 18] +(106) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#105] Right keys [1]: [d_date_sk#111] Join type: Inner Join condition: None -(107) Project [codegen id : 18] +(107) Project [codegen id : 6] Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Input [11]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] @@ -688,14 +688,14 @@ Arguments: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#127 ASC NULLS FIRST], output=[prev_year#123,year#124,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#125,curr_yr_cnt#126,sales_cnt_diff#127,sales_amt_diff#128]), [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128], 100, 0, [sales_cnt_diff#127 ASC NULLS FIRST], [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] -(127) CometColumnarToRow [codegen id : 19] +(127) CometNativeColumnarToRow Input [10]: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (131) -+- * CometColumnarToRow (130) ++- CometNativeColumnarToRow (130) +- CometFilter (129) +- CometNativeScan parquet spark_catalog.default.date_dim (128) @@ -711,7 +711,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (131) BroadcastExchange @@ -724,7 +724,7 @@ Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN d Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 BroadcastExchange (135) -+- * CometColumnarToRow (134) ++- CometNativeColumnarToRow (134) +- CometFilter (133) +- CometNativeScan parquet spark_catalog.default.date_dim (132) @@ -740,7 +740,7 @@ ReadSchema: struct Input [2]: [d_date_sk#75, d_year#76] Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) -(134) CometColumnarToRow [codegen id : 1] +(134) CometNativeColumnarToRow Input [2]: [d_date_sk#75, d_year#76] (135) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt index 9f6954be38..42106a0ffd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometSortMergeJoin @@ -25,16 +25,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -55,12 +55,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -81,12 +81,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -117,16 +117,16 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -147,12 +147,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -173,12 +173,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/simplified.txt index aaf98f4b09..84b015b466 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/simplified.txt @@ -1,173 +1,165 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - 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 [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 [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] +CometNativeColumnarToRow + CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + 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 [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 [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] - 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] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometExchange [cr_order_number,cr_item_sk] #7 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (2) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometExchange [sr_ticket_number,sr_item_sk] #9 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (3) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometExchange [wr_order_number,wr_item_sk] #11 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 + 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] #14 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] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (4) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (5) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (6) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt index 02ac3ecdc2..e8f4b46033 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt index d0520c8b63..0e2370eea4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt index 02ac3ecdc2..e8f4b46033 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange 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 d0520c8b63..0e2370eea4 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 @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/explain.txt index fab101ea7c..c52cd3ef9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometTakeOrderedAndProject (34) +- CometHashAggregate (33) +- CometExchange (32) @@ -195,6 +195,6 @@ Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/extended.txt index cbb627c528..cfa19a28b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/simplified.txt index abb482c3d5..f502dc6f07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/simplified.txt @@ -1,37 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - 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 [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] - CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] - CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedExchange [i_item_sk,i_category] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] - CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedExchange [i_item_sk,i_category] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #3 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + 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 [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] + CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometProject [i_category] [i_item_sk,i_category] + CometFilter [i_item_sk,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/explain.txt index ae530b4900..d68b43b40a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (92) +- * HashAggregate (91) - +- * CometColumnarToRow (90) + +- CometNativeColumnarToRow (90) +- CometColumnarExchange (89) +- * HashAggregate (88) +- * Expand (87) @@ -9,7 +9,7 @@ TakeOrderedAndProject (92) :- * Project (32) : +- * BroadcastHashJoin LeftOuter BuildRight (31) : :- * HashAggregate (16) - : : +- * CometColumnarToRow (15) + : : +- CometNativeColumnarToRow (15) : : +- CometColumnarExchange (14) : : +- * HashAggregate (13) : : +- * Project (12) @@ -21,12 +21,12 @@ TakeOrderedAndProject (92) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (10) - : : +- * CometColumnarToRow (9) + : : +- CometNativeColumnarToRow (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (30) : +- * HashAggregate (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometColumnarExchange (27) : +- * HashAggregate (26) : +- * Project (25) @@ -42,7 +42,7 @@ TakeOrderedAndProject (92) : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) : :- BroadcastExchange (42) : : +- * HashAggregate (41) - : : +- * CometColumnarToRow (40) + : : +- CometNativeColumnarToRow (40) : : +- CometColumnarExchange (39) : : +- * HashAggregate (38) : : +- * Project (37) @@ -51,7 +51,7 @@ TakeOrderedAndProject (92) : : : +- Scan parquet spark_catalog.default.catalog_sales (33) : : +- ReusedExchange (35) : +- * HashAggregate (51) - : +- * CometColumnarToRow (50) + : +- CometNativeColumnarToRow (50) : +- CometColumnarExchange (49) : +- * HashAggregate (48) : +- * Project (47) @@ -62,7 +62,7 @@ TakeOrderedAndProject (92) +- * Project (85) +- * BroadcastHashJoin LeftOuter BuildRight (84) :- * HashAggregate (69) - : +- * CometColumnarToRow (68) + : +- CometNativeColumnarToRow (68) : +- CometColumnarExchange (67) : +- * HashAggregate (66) : +- * Project (65) @@ -74,12 +74,12 @@ TakeOrderedAndProject (92) : : : +- Scan parquet spark_catalog.default.web_sales (54) : : +- ReusedExchange (57) : +- BroadcastExchange (63) - : +- * CometColumnarToRow (62) + : +- CometNativeColumnarToRow (62) : +- CometFilter (61) : +- CometNativeScan parquet spark_catalog.default.web_page (60) +- BroadcastExchange (83) +- * HashAggregate (82) - +- * CometColumnarToRow (81) + +- CometNativeColumnarToRow (81) +- CometColumnarExchange (80) +- * HashAggregate (79) +- * Project (78) @@ -101,23 +101,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] @@ -132,24 +132,24 @@ ReadSchema: struct Input [1]: [s_store_sk#7] Condition : isnotnull(s_store_sk#7) -(9) CometColumnarToRow [codegen id : 2] +(9) CometNativeColumnarToRow Input [1]: [s_store_sk#7] (10) BroadcastExchange Input [1]: [s_store_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(12) Project [codegen id : 3] +(12) Project [codegen id : 1] Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Keys [1]: [s_store_sk#7] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] @@ -160,10 +160,10 @@ Results [3]: [s_store_sk#7, sum#10, sum#11] Input [3]: [s_store_sk#7, sum#10, sum#11] Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(15) CometColumnarToRow [codegen id : 8] +(15) CometNativeColumnarToRow Input [3]: [s_store_sk#7, sum#10, sum#11] -(16) HashAggregate [codegen id : 8] +(16) HashAggregate [codegen id : 4] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] @@ -178,40 +178,40 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Condition : isnotnull(sr_store_sk#16) (20) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#20] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] (23) ReusedExchange [Reuses operator id: 10] Output [1]: [s_store_sk#21] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_store_sk#16] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Keys [1]: [s_store_sk#21] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] @@ -222,10 +222,10 @@ Results [3]: [s_store_sk#21, sum#24, sum#25] Input [3]: [s_store_sk#21, sum#24, sum#25] Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#21, sum#24, sum#25] -(29) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 3] Input [3]: [s_store_sk#21, sum#24, sum#25] Keys [1]: [s_store_sk#21] Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] @@ -236,13 +236,13 @@ Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26 Input [3]: [s_store_sk#21, returns#28, profit_loss#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#7] Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 4] Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33] Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] @@ -253,23 +253,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 5] Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] (35) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#38] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#37] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 5] Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] -(38) HashAggregate [codegen id : 10] +(38) HashAggregate [codegen id : 5] Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Keys [1]: [cs_call_center_sk#34] Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] @@ -280,10 +280,10 @@ Results [3]: [cs_call_center_sk#34, sum#41, sum#42] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometColumnarToRow [codegen id : 11] +(40) CometNativeColumnarToRow Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 6] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Keys [1]: [cs_call_center_sk#34] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] @@ -301,23 +301,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] +(44) ColumnarToRow [codegen id : 7] Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] (45) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#50] -(46) BroadcastHashJoin [codegen id : 13] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cr_returned_date_sk#49] Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(47) Project [codegen id : 13] +(47) Project [codegen id : 7] Output [2]: [cr_return_amount#47, cr_net_loss#48] Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] -(48) HashAggregate [codegen id : 13] +(48) HashAggregate [codegen id : 7] Input [2]: [cr_return_amount#47, cr_net_loss#48] Keys: [] Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] @@ -328,7 +328,7 @@ Results [2]: [sum#53, sum#54] Input [2]: [sum#53, sum#54] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) CometColumnarToRow +(50) CometNativeColumnarToRow Input [2]: [sum#53, sum#54] (51) HashAggregate @@ -338,11 +338,11 @@ Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_ne Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] -(52) BroadcastNestedLoopJoin [codegen id : 14] +(52) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 8] Output [5]: [sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61] Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] @@ -354,23 +354,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 17] +(55) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -(56) Filter [codegen id : 17] +(56) Filter [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] Condition : isnotnull(ws_web_page_sk#62) (57) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#66] -(58) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#65] Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(59) Project [codegen id : 9] Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] @@ -385,24 +385,24 @@ ReadSchema: struct Input [1]: [wp_web_page_sk#67] Condition : isnotnull(wp_web_page_sk#67) -(62) CometColumnarToRow [codegen id : 16] +(62) CometNativeColumnarToRow Input [1]: [wp_web_page_sk#67] (63) BroadcastExchange Input [1]: [wp_web_page_sk#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(64) BroadcastHashJoin [codegen id : 17] +(64) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_web_page_sk#62] Right keys [1]: [wp_web_page_sk#67] Join type: Inner Join condition: None -(65) Project [codegen id : 17] +(65) Project [codegen id : 9] Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 9] Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Keys [1]: [wp_web_page_sk#67] Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] @@ -413,10 +413,10 @@ Results [3]: [wp_web_page_sk#67, sum#70, sum#71] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometColumnarToRow [codegen id : 22] +(68) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -(69) HashAggregate [codegen id : 22] +(69) HashAggregate [codegen id : 12] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Keys [1]: [wp_web_page_sk#67] Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] @@ -431,40 +431,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -(72) Filter [codegen id : 20] +(72) Filter [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] Condition : isnotnull(wr_web_page_sk#76) (73) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#80] -(74) BroadcastHashJoin [codegen id : 20] +(74) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_returned_date_sk#79] Right keys [1]: [d_date_sk#80] Join type: Inner Join condition: None -(75) Project [codegen id : 20] +(75) Project [codegen id : 10] Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] (76) ReusedExchange [Reuses operator id: 63] Output [1]: [wp_web_page_sk#81] -(77) BroadcastHashJoin [codegen id : 20] +(77) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_web_page_sk#76] Right keys [1]: [wp_web_page_sk#81] Join type: Inner Join condition: None -(78) Project [codegen id : 20] +(78) Project [codegen id : 10] Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -(79) HashAggregate [codegen id : 20] +(79) HashAggregate [codegen id : 10] Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Keys [1]: [wp_web_page_sk#81] Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] @@ -475,10 +475,10 @@ Results [3]: [wp_web_page_sk#81, sum#84, sum#85] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(81) CometColumnarToRow [codegen id : 21] +(81) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -(82) HashAggregate [codegen id : 21] +(82) HashAggregate [codegen id : 11] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Keys [1]: [wp_web_page_sk#81] Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] @@ -489,23 +489,23 @@ Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77) Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 22] +(84) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wp_web_page_sk#67] Right keys [1]: [wp_web_page_sk#81] Join type: LeftOuter Join condition: None -(85) Project [codegen id : 22] +(85) Project [codegen id : 12] Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93] Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] (86) Union -(87) Expand [codegen id : 23] +(87) Expand [codegen id : 13] Input [5]: [sales#14, returns#30, profit#31, channel#32, id#33] Arguments: [[sales#14, returns#30, profit#31, channel#32, id#33, 0], [sales#14, returns#30, profit#31, channel#32, null, 1], [sales#14, returns#30, profit#31, null, null, 3]], [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] -(88) HashAggregate [codegen id : 23] +(88) HashAggregate [codegen id : 13] Input [6]: [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] Keys [3]: [channel#94, id#95, spark_grouping_id#96] Functions [3]: [partial_sum(sales#14), partial_sum(returns#30), partial_sum(profit#31)] @@ -516,10 +516,10 @@ Results [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] Arguments: hashpartitioning(channel#94, id#95, spark_grouping_id#96, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(90) CometColumnarToRow [codegen id : 24] +(90) CometNativeColumnarToRow Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -(91) HashAggregate [codegen id : 24] +(91) HashAggregate [codegen id : 14] Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] Keys [3]: [channel#94, id#95, spark_grouping_id#96] Functions [3]: [sum(sales#14), sum(returns#30), sum(profit#31)] @@ -534,7 +534,7 @@ Arguments: 100, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (97) -+- * CometColumnarToRow (96) ++- CometNativeColumnarToRow (96) +- CometProject (95) +- CometFilter (94) +- CometNativeScan parquet spark_catalog.default.date_dim (93) @@ -555,7 +555,7 @@ Condition : (((isnotnull(d_date#115) AND (d_date#115 >= 2000-08-03)) AND (d_date Input [2]: [d_date_sk#6, d_date#115] Arguments: [d_date_sk#6], [d_date_sk#6] -(96) CometColumnarToRow [codegen id : 1] +(96) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (97) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt index d12b8dde24..5d2ca1a9d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -8,7 +8,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -20,22 +20,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -47,19 +47,19 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -68,12 +68,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -82,14 +82,14 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- BroadcastHashJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -101,17 +101,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -123,12 +123,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/simplified.txt index 4256e90759..5a732180f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/simplified.txt @@ -1,22 +1,22 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) + WholeStageCodegen (14) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) + WholeStageCodegen (13) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (8) + WholeStageCodegen (4) Project [sales,returns,profit,profit_loss,s_store_sk] BroadcastHashJoin [s_store_sk,s_store_sk] HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_sk] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] Project [ss_ext_sales_price,ss_net_profit,s_store_sk] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -28,29 +28,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_sk] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] Project [sr_return_amt,sr_net_loss,s_store_sk] BroadcastHashJoin [sr_store_sk,s_store_sk] @@ -65,17 +61,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [sales,returns,profit,profit_loss,cs_call_center_sk] BroadcastNestedLoopJoin InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) + WholeStageCodegen (6) HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cs_call_center_sk] #8 - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -86,10 +82,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [d_date_sk] #3 HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] Project [cr_return_amount,cr_net_loss] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] @@ -99,14 +95,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) + WholeStageCodegen (12) Project [sales,returns,profit,profit_loss,wp_web_page_sk] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wp_web_page_sk] #10 - WholeStageCodegen (17) + WholeStageCodegen (9) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] @@ -121,19 +117,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometNativeColumnarToRow + CometFilter [wp_web_page_sk] + CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter BroadcastExchange #12 - WholeStageCodegen (21) + WholeStageCodegen (11) HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wp_web_page_sk] #13 - WholeStageCodegen (20) + WholeStageCodegen (10) HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] Project [wr_return_amt,wr_net_loss,wp_web_page_sk] BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt index bb7ed0a50c..9155629835 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt @@ -477,7 +477,7 @@ Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (90) -+- * CometColumnarToRow (89) ++- CometNativeColumnarToRow (89) +- CometProject (88) +- CometFilter (87) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) @@ -498,7 +498,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(89) CometColumnarToRow [codegen id : 1] +(89) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (90) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt index ed8a9e38ca..a16105c248 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt index a1243769e5..94722e1db2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt @@ -25,12 +25,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt index bb7ed0a50c..9155629835 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt @@ -477,7 +477,7 @@ Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (90) -+- * CometColumnarToRow (89) ++- CometNativeColumnarToRow (89) +- CometProject (88) +- CometFilter (87) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) @@ -498,7 +498,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(89) CometColumnarToRow [codegen id : 1] +(89) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (90) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt index ed8a9e38ca..a16105c248 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a1243769e5..94722e1db2 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 @@ -25,12 +25,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/explain.txt index 4dc7ac380a..928636dfa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (70) +- * Project (69) - +- * CometColumnarToRow (68) + +- CometNativeColumnarToRow (68) +- CometSortMergeJoin (67) :- CometProject (45) : +- CometSortMergeJoin (44) @@ -372,7 +372,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner -(68) CometColumnarToRow [codegen id : 4] +(68) CometNativeColumnarToRow Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] (69) Project [codegen id : 4] @@ -387,7 +387,7 @@ Arguments: 100, [ratio#61 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -403,7 +403,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (74) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt index 65256f2124..af4b7000b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSortMergeJoin :- CometProject : +- CometSortMergeJoin @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/simplified.txt index ebd0840eec..5419c4229c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (4) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] @@ -24,11 +24,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt index 3d3ea5ab86..e4118a724e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt index 77c05217f4..67056a3026 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt @@ -21,11 +21,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt index 3d3ea5ab86..e4118a724e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort 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 77c05217f4..67056a3026 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 @@ -21,11 +21,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/explain.txt index e51e7eb2af..9481c84819 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (32) +- * Project (31) +- * BroadcastHashJoin Inner BuildRight (30) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -17,17 +17,17 @@ TakeOrderedAndProject (32) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.customer (25) @@ -41,23 +41,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#10] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] @@ -76,20 +76,20 @@ Condition : (((isnotnull(s_number_employees#12) AND (s_number_employees#12 >= 20 Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] Arguments: [s_store_sk#11, s_city#13], [s_store_sk#11, s_city#13] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [s_store_sk#11, s_city#13] (11) BroadcastExchange Input [2]: [s_store_sk#11, s_city#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11, s_city#13] @@ -108,24 +108,24 @@ Condition : (((hd_dep_count#15 = 6) OR (hd_vehicle_count#16 > 2)) AND isnotnull( Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] Arguments: [hd_demo_sk#14], [hd_demo_sk#14] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#14] (18) BroadcastExchange Input [1]: [hd_demo_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13, hd_demo_sk#14] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] @@ -136,10 +136,10 @@ Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] @@ -161,20 +161,20 @@ Condition : isnotnull(c_customer_sk#25) Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] Arguments: [c_customer_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#26, 20)) AS c_first_name#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#27, 30)) AS c_last_name#29] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] (29) BroadcastExchange Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [7]: [c_last_name#29, c_first_name#28, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#13] Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#28, c_last_name#29] @@ -186,7 +186,7 @@ Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((isnotnull(d_dow#32) AND (d_dow#32 = 1)) AND d_year#31 IN (1999,20 Input [3]: [d_date_sk#10, d_year#31, d_dow#32] Arguments: [d_date_sk#10], [d_date_sk#10] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt index 05bd194c34..f24d730d8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject +- Project +- BroadcastHashJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -16,27 +16,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/simplified.txt index 1ee7a286a4..48b20af3a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/simplified.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (6) + WholeStageCodegen (2) Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] @@ -20,35 +20,27 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + CometNativeColumnarToRow + CometProject [s_store_sk,s_city] + CometFilter [s_store_sk,s_number_employees,s_city] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt index 16bf78be93..23afa9a4ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt @@ -181,7 +181,7 @@ Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt index 012403275a..871cf47610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt index 5b252a906b..d9d0bedcfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt index 16bf78be93..23afa9a4ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt @@ -181,7 +181,7 @@ Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt index 012403275a..871cf47610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 5b252a906b..d9d0bedcfb 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 @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/explain.txt index 225159b9e0..a14bc883e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (44) +- * HashAggregate (43) - +- * CometColumnarToRow (42) + +- CometNativeColumnarToRow (42) +- CometColumnarExchange (41) +- * HashAggregate (40) +- * Project (39) @@ -15,12 +15,12 @@ TakeOrderedAndProject (44) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.store (7) +- BroadcastExchange (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometHashAggregate (35) +- CometExchange (34) +- CometHashAggregate (33) @@ -53,23 +53,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -88,20 +88,20 @@ Condition : (isnotnull(s_store_sk#6) AND isnotnull(static_invoke(CharVarcharCode Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] Arguments: [s_store_sk#6, s_store_name#7, s_zip#9], [s_store_sk#6, s_store_name#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#8, 10)) AS s_zip#9] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] (11) BroadcastExchange Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#9] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#9] @@ -207,24 +207,24 @@ Input [1]: [ca_zip#11] Keys [1]: [ca_zip#11] Functions: [] -(36) CometColumnarToRow [codegen id : 3] +(36) CometNativeColumnarToRow Input [1]: [ca_zip#11] (37) BroadcastExchange Input [1]: [ca_zip#11] Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 4] +(38) BroadcastHashJoin [codegen id : 1] Left keys [1]: [substr(s_zip#9, 1, 2)] Right keys [1]: [substr(ca_zip#11, 1, 2)] Join type: Inner Join condition: None -(39) Project [codegen id : 4] +(39) Project [codegen id : 1] Output [2]: [ss_net_profit#2, s_store_name#7] Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#9, ca_zip#11] -(40) HashAggregate [codegen id : 4] +(40) HashAggregate [codegen id : 1] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] @@ -235,10 +235,10 @@ Results [2]: [s_store_name#7, sum#21] Input [2]: [s_store_name#7, sum#21] Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(42) CometColumnarToRow [codegen id : 5] +(42) CometNativeColumnarToRow Input [2]: [s_store_name#7, sum#21] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 2] Input [2]: [s_store_name#7, sum#21] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] @@ -253,7 +253,7 @@ Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometNativeScan parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2) Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt index 257b7f0d77..8c9eb7b8c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -14,22 +14,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/simplified.txt index 4df6d8f659..97b1393e3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] BroadcastHashJoin [s_zip,ca_zip] @@ -18,46 +18,40 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometNativeColumnarToRow + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter + CometNativeColumnarToRow + CometHashAggregate [ca_zip] + CometExchange [ca_zip] #5 CometHashAggregate [ca_zip] - CometExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #6 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #7 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #8 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometBroadcastHashJoin [ca_zip,ca_zip] + CometProject [ca_zip] [ca_zip] + CometFilter [ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_zip] + CometBroadcastExchange [ca_zip] #6 + CometProject [ca_zip] + CometFilter [ca_zip,cnt] + CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] + CometExchange [ca_zip] #7 + CometHashAggregate [ca_zip,count] + CometProject [ca_zip] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometProject [ca_zip] [ca_address_sk,ca_zip] + CometFilter [ca_address_sk,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [c_current_addr_sk] #8 + CometProject [c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt index 1b9324fe93..58bc58a57c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt @@ -253,7 +253,7 @@ Input [2]: [s_store_name#9, sum(ss_net_profit)#23] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt index 8e04bbeba7..4d7a7d1ce7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt index d1542ab607..fc8379fb0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt index 1b9324fe93..58bc58a57c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt @@ -253,7 +253,7 @@ Input [2]: [s_store_name#9, sum(ss_net_profit)#23] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt index 8e04bbeba7..4d7a7d1ce7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d1542ab607..fc8379fb0d 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 @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/explain.txt index 76d323f165..62d37a7896 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (106) +CometNativeColumnarToRow (106) +- CometTakeOrderedAndProject (105) +- CometHashAggregate (104) +- CometExchange (103) @@ -581,14 +581,14 @@ Functions [3]: [sum(sales#74), sum(returns#75), sum(profit#76)] Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,id#90 ASC NULLS FIRST], output=[channel#89,id#90,sales#98,returns#99,profit#100]), [channel#89, id#90, sales#98, returns#99, profit#100], 100, 0, [channel#89 ASC NULLS FIRST, id#90 ASC NULLS FIRST], [channel#89, id#90, sales#98, returns#99, profit#100] -(106) CometColumnarToRow [codegen id : 4] +(106) CometNativeColumnarToRow Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (111) -+- * CometColumnarToRow (110) ++- CometNativeColumnarToRow (110) +- CometProject (109) +- CometFilter (108) +- CometNativeScan parquet spark_catalog.default.date_dim (107) @@ -609,7 +609,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(110) CometColumnarToRow [codegen id : 1] +(110) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (111) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt index 9d97e27318..6853afccad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/simplified.txt index 09e9224cef..3d30801890 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/simplified.txt @@ -1,124 +1,120 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - 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] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - 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] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #11 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - 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] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #15 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] + CometExchange [s_store_id] #2 + 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] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 + WholeStageCodegen (1) + Filter [ss_store_sk,ss_item_sk,ss_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #5 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #10 + 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] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #11 + WholeStageCodegen (2) + Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] + CometExchange [web_site_id] #14 + 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] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #15 + WholeStageCodegen (3) + Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt index 223374746d..35f65c3d90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt @@ -576,7 +576,7 @@ Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -597,7 +597,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt index ae6fc19b60..dd2aed67bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt index 198646e0fd..9781eeab1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #5 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt index 223374746d..35f65c3d90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt @@ -576,7 +576,7 @@ Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -597,7 +597,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt index ae6fc19b60..dd2aed67bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 198646e0fd..9781eeab1c 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 @@ -26,12 +26,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #5 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/explain.txt index bcc4d0f4a0..889cd9e747 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (54) : : +- * BroadcastHashJoin Inner BuildRight (38) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) @@ -20,18 +20,18 @@ TakeOrderedAndProject (54) : : : : : +- Scan parquet spark_catalog.default.catalog_returns (1) : : : : +- ReusedExchange (4) : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) + : : : +- CometNativeColumnarToRow (10) : : : +- CometProject (9) : : : +- CometFilter (8) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : : +- BroadcastExchange (37) : : +- * Filter (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) + : : +- CometNativeColumnarToRow (30) : : +- CometColumnarExchange (29) : : +- * HashAggregate (28) : : +- * Project (27) @@ -44,12 +44,12 @@ TakeOrderedAndProject (54) : : : +- ReusedExchange (22) : : +- ReusedExchange (25) : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometProject (42) : +- CometFilter (41) : +- CometNativeScan parquet spark_catalog.default.customer (40) +- BroadcastExchange (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometNativeScan parquet spark_catalog.default.customer_address (47) @@ -63,23 +63,23 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] @@ -98,24 +98,24 @@ Condition : (isnotnull(ca_address_sk#7) AND isnotnull(static_invoke(CharVarcharC Input [2]: [ca_address_sk#7, ca_state#8] Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)) AS ca_state#9] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [ca_address_sk#7, ca_state#9] (11) BroadcastExchange Input [2]: [ca_address_sk#7, ca_state#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#9] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] @@ -126,17 +126,17 @@ Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 11] +(16) CometNativeColumnarToRow Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -(17) HashAggregate [codegen id : 11] +(17) HashAggregate [codegen id : 5] Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] -(18) Filter [codegen id : 11] +(18) Filter [codegen id : 5] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) @@ -148,40 +148,40 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#19), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 2] Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -(21) Filter [codegen id : 6] +(21) Filter [codegen id : 2] Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] Condition : isnotnull(cr_returning_addr_sk#17) (22) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(24) Project [codegen id : 2] Output [3]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19, d_date_sk#20] (25) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#21, ca_state#22] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returning_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 2] Output [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#22] Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, ca_address_sk#21, ca_state#22] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#22] Keys [2]: [cr_returning_customer_sk#16, ca_state#22] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#18))] @@ -192,17 +192,17 @@ Results [3]: [cr_returning_customer_sk#16, ca_state#22, sum#24] Input [3]: [cr_returning_customer_sk#16, ca_state#22, sum#24] Arguments: hashpartitioning(cr_returning_customer_sk#16, ca_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [3]: [cr_returning_customer_sk#16, ca_state#22, sum#24] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [3]: [cr_returning_customer_sk#16, ca_state#22, sum#24] Keys [2]: [cr_returning_customer_sk#16, ca_state#22] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))#12] Results [2]: [ca_state#22 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#18))#12,17,2) AS ctr_total_return#26] -(32) HashAggregate [codegen id : 7] +(32) HashAggregate [codegen id : 3] Input [2]: [ctr_state#25, ctr_total_return#26] Keys [1]: [ctr_state#25] Functions [1]: [partial_avg(ctr_total_return#26)] @@ -213,17 +213,17 @@ Results [3]: [ctr_state#25, sum#29, count#30] Input [3]: [ctr_state#25, sum#29, count#30] Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 8] +(34) CometNativeColumnarToRow Input [3]: [ctr_state#25, sum#29, count#30] -(35) HashAggregate [codegen id : 8] +(35) HashAggregate [codegen id : 4] Input [3]: [ctr_state#25, sum#29, count#30] Keys [1]: [ctr_state#25] Functions [1]: [avg(ctr_total_return#26)] Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] -(36) Filter [codegen id : 8] +(36) Filter [codegen id : 4] Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) @@ -231,13 +231,13 @@ Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 11] +(38) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_state#14] Right keys [1]: [ctr_state#25] Join type: Inner Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) -(39) Project [codegen id : 11] +(39) Project [codegen id : 5] Output [2]: [ctr_customer_sk#13, ctr_total_return#15] Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] @@ -256,20 +256,20 @@ Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] Arguments: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42], [c_customer_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#34, 16)) AS c_customer_id#39, c_current_addr_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#36, 10)) AS c_salutation#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#42] -(43) CometColumnarToRow [codegen id : 9] +(43) CometNativeColumnarToRow Input [6]: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] (44) BroadcastExchange Input [6]: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_customer_sk#13] Right keys [1]: [c_customer_sk#33] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(46) Project [codegen id : 5] Output [6]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] Input [8]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] @@ -288,20 +288,20 @@ Condition : ((isnotnull(ca_state#50) AND (static_invoke(CharVarcharCodegenUtils. Input [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] Arguments: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60], [ca_address_sk#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#44, 10)) AS ca_street_number#55, ca_street_name#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_type#46, 15)) AS ca_street_type#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_suite_number#47, 10)) AS ca_suite_number#57, ca_city#48, ca_county#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#50, 2)) AS ca_state#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#51, 10)) AS ca_zip#59, ca_country#52, ca_gmt_offset#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_location_type#54, 20)) AS ca_location_type#60] -(50) CometColumnarToRow [codegen id : 10] +(50) CometNativeColumnarToRow Input [12]: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] (51) BroadcastExchange Input [12]: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#35] Right keys [1]: [ca_address_sk#43] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(53) Project [codegen id : 5] Output [16]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] Input [18]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42, ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] @@ -313,7 +313,7 @@ Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIR Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (59) -+- * CometColumnarToRow (58) ++- CometNativeColumnarToRow (58) +- CometProject (57) +- CometFilter (56) +- CometNativeScan parquet spark_catalog.default.date_dim (55) @@ -334,7 +334,7 @@ Condition : ((isnotnull(d_year#61) AND (d_year#61 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#6, d_year#61] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(58) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (59) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt index 0e70ec42fa..841b2ca52d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -19,28 +19,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -52,22 +52,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/simplified.txt index da396ce8a9..fe57ceb6af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (5) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] @@ -8,10 +8,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] @@ -23,37 +23,33 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st Scan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (8) + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] @@ -70,17 +66,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometNativeColumnarToRow + CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt index 492a321f97..ace4d64536 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt @@ -285,7 +285,7 @@ Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt index 9708802e79..731fcea912 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt index 1091e272cc..120f97f276 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt index 492a321f97..ace4d64536 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt @@ -285,7 +285,7 @@ Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt index 9708802e79..731fcea912 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 1091e272cc..120f97f276 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/explain.txt index 6ff1da2cc8..17c90a1007 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (26) +CometNativeColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometColumnarExchange (23) @@ -11,7 +11,7 @@ : +- * BroadcastHashJoin Inner BuildRight (13) : :- * Project (11) : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -21,7 +21,7 @@ : : +- * ColumnarToRow (6) : : +- Scan parquet spark_catalog.default.inventory (5) : +- ReusedExchange (12) - +- * CometColumnarToRow (19) + +- CometNativeColumnarToRow (19) +- CometProject (18) +- CometFilter (17) +- CometNativeScan parquet spark_catalog.default.store_sales (16) @@ -42,7 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#2, 16)) AS i_item_id#6, i_item_desc#3, i_current_price#4] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] (5) Scan parquet spark_catalog.default.inventory @@ -68,26 +68,26 @@ Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] Input [2]: [inv_item_sk#7, inv_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [inv_item_sk#7] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] (12) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#11] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] @@ -110,20 +110,20 @@ Condition : isnotnull(ss_item_sk#12) Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] Arguments: [ss_item_sk#12], [ss_item_sk#12] -(19) CometColumnarToRow +(19) CometNativeColumnarToRow Input [1]: [ss_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#12] Join type: Inner Join condition: None -(21) Project [codegen id : 4] +(21) Project [codegen id : 3] Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#12] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 3] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Functions: [] @@ -143,14 +143,14 @@ Functions: [] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] -(26) CometColumnarToRow [codegen id : 5] +(26) CometNativeColumnarToRow Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.date_dim (27) @@ -171,7 +171,7 @@ Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#14] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt index c86e7ba2cb..8abf5c1fb1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -10,7 +10,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -21,16 +21,16 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/simplified.txt index b60bf92c5b..b8b1c8b053 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/simplified.txt @@ -1,45 +1,41 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,ss_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [inv_item_sk,inv_date_sk] + Filter [inv_quantity_on_hand,inv_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + CometNativeColumnarToRow + CometProject [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/explain.txt index d61d464b98..677ff783cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt index 9fbb7893b8..2ab6dd2d0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/simplified.txt index 28d160a4a8..b92f067e53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt index d61d464b98..677ff783cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/extended.txt index 9fbb7893b8..2ab6dd2d0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt index 28d160a4a8..b92f067e53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/explain.txt index 5b445f714a..7b43bd9670 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (50) :- * Project (33) : +- * BroadcastHashJoin Inner BuildRight (32) : :- * HashAggregate (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- * HashAggregate (14) : : +- * Project (13) @@ -16,14 +16,14 @@ TakeOrderedAndProject (50) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_returns (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : +- ReusedExchange (11) : +- BroadcastExchange (31) : +- * HashAggregate (30) - : +- * CometColumnarToRow (29) + : +- CometNativeColumnarToRow (29) : +- CometColumnarExchange (28) : +- * HashAggregate (27) : +- * Project (26) @@ -37,7 +37,7 @@ TakeOrderedAndProject (50) : +- ReusedExchange (24) +- BroadcastExchange (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * Project (42) @@ -59,10 +59,10 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) @@ -81,37 +81,37 @@ Condition : (isnotnull(i_item_sk#5) AND isnotnull(static_invoke(CharVarcharCodeg Input [2]: [i_item_sk#5, i_item_id#6] Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#7] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [i_item_sk#5, i_item_id#7] (8) BroadcastExchange Input [2]: [i_item_sk#5, i_item_id#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [sr_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] (11) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#8] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [sr_returned_date_sk#3] Right keys [1]: [d_date_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [2]: [sr_return_quantity#2, i_item_id#7] Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [2]: [sr_return_quantity#2, i_item_id#7] Keys [1]: [i_item_id#7] Functions [1]: [partial_sum(sr_return_quantity#2)] @@ -122,10 +122,10 @@ Results [2]: [i_item_id#7, sum#10] Input [2]: [i_item_id#7, sum#10] Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 12] +(16) CometNativeColumnarToRow Input [2]: [i_item_id#7, sum#10] -(17) HashAggregate [codegen id : 12] +(17) HashAggregate [codegen id : 6] Input [2]: [i_item_id#7, sum#10] Keys [1]: [i_item_id#7] Functions [1]: [sum(sr_return_quantity#2)] @@ -140,40 +140,40 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#16), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 2] Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -(20) Filter [codegen id : 6] +(20) Filter [codegen id : 2] Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] Condition : isnotnull(cr_item_sk#14) (21) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#17, i_item_id#18] -(22) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_item_sk#14] Right keys [1]: [i_item_sk#17] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(23) Project [codegen id : 2] Output [3]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#18] Input [5]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16, i_item_sk#17, i_item_id#18] (24) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#19] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returned_date_sk#16] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(26) Project [codegen id : 2] Output [2]: [cr_return_quantity#15, i_item_id#18] Input [4]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#18, d_date_sk#19] -(27) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 2] Input [2]: [cr_return_quantity#15, i_item_id#18] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(cr_return_quantity#15)] @@ -184,10 +184,10 @@ Results [2]: [i_item_id#18, sum#21] Input [2]: [i_item_id#18, sum#21] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(29) CometColumnarToRow [codegen id : 7] +(29) CometNativeColumnarToRow Input [2]: [i_item_id#18, sum#21] -(30) HashAggregate [codegen id : 7] +(30) HashAggregate [codegen id : 3] Input [2]: [i_item_id#18, sum#21] Keys [1]: [i_item_id#18] Functions [1]: [sum(cr_return_quantity#15)] @@ -198,13 +198,13 @@ Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#15)#22 AS cr_it Input [2]: [item_id#23, cr_item_qty#24] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 12] +(32) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#23] Join type: Inner Join condition: None -(33) Project [codegen id : 12] +(33) Project [codegen id : 6] Output [3]: [item_id#12, sr_item_qty#13, cr_item_qty#24] Input [4]: [item_id#12, sr_item_qty#13, item_id#23, cr_item_qty#24] @@ -216,40 +216,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] +(35) ColumnarToRow [codegen id : 4] Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -(36) Filter [codegen id : 10] +(36) Filter [codegen id : 4] Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] Condition : isnotnull(wr_item_sk#25) (37) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#28, i_item_id#29] -(38) BroadcastHashJoin [codegen id : 10] +(38) BroadcastHashJoin [codegen id : 4] Left keys [1]: [wr_item_sk#25] Right keys [1]: [i_item_sk#28] Join type: Inner Join condition: None -(39) Project [codegen id : 10] +(39) Project [codegen id : 4] Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#29] Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#28, i_item_id#29] (40) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#30] -(41) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [wr_returned_date_sk#27] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 10] +(42) Project [codegen id : 4] Output [2]: [wr_return_quantity#26, i_item_id#29] Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#29, d_date_sk#30] -(43) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 4] Input [2]: [wr_return_quantity#26, i_item_id#29] Keys [1]: [i_item_id#29] Functions [1]: [partial_sum(wr_return_quantity#26)] @@ -260,10 +260,10 @@ Results [2]: [i_item_id#29, sum#32] Input [2]: [i_item_id#29, sum#32] Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 11] +(45) CometNativeColumnarToRow Input [2]: [i_item_id#29, sum#32] -(46) HashAggregate [codegen id : 11] +(46) HashAggregate [codegen id : 5] Input [2]: [i_item_id#29, sum#32] Keys [1]: [i_item_id#29] Functions [1]: [sum(wr_return_quantity#26)] @@ -274,13 +274,13 @@ Results [2]: [i_item_id#29 AS item_id#34, sum(wr_return_quantity#26)#33 AS wr_it Input [2]: [item_id#34, wr_item_qty#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 12] +(48) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#34] Join type: Inner Join condition: None -(49) Project [codegen id : 12] +(49) Project [codegen id : 6] Output [8]: [item_id#12, sr_item_qty#13, (((cast(sr_item_qty#13 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS sr_dev#36, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS cr_dev#37, wr_item_qty#35, (((cast(wr_item_qty#35 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS wr_dev#38, (cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as decimal(20,0)) / 3.0) AS average#39] Input [5]: [item_id#12, sr_item_qty#13, cr_item_qty#24, item_id#34, wr_item_qty#35] @@ -292,7 +292,7 @@ Arguments: 100, [item_id#12 ASC NULLS FIRST, sr_item_qty#13 ASC NULLS FIRST], [i Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (64) -+- * CometColumnarToRow (63) ++- CometNativeColumnarToRow (63) +- CometProject (62) +- CometBroadcastHashJoin (61) :- CometFilter (52) @@ -365,7 +365,7 @@ Arguments: [d_date#40], [d_date#41], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#40] Arguments: [d_date_sk#8], [d_date_sk#8] -(63) CometColumnarToRow [codegen id : 1] +(63) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (64) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt index 1cb87c7eb1..7616ffd2b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -30,12 +30,12 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter @@ -50,7 +50,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -62,12 +62,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -82,7 +82,7 @@ TakeOrderedAndProject : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -94,12 +94,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/simplified.txt index 0673c590f5..53a9745325 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - WholeStageCodegen (12) + WholeStageCodegen (6) Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] BroadcastHashJoin [item_id,item_id] Project [item_id,sr_item_qty,cr_item_qty] BroadcastHashJoin [item_id,item_id] HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,sr_return_quantity] [sum,sum] Project [sr_return_quantity,i_item_id] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -20,39 +20,35 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,cr_return_quantity] [sum,sum] Project [cr_return_quantity,i_item_id] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] @@ -69,12 +65,12 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (5) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #9 - WholeStageCodegen (10) + WholeStageCodegen (4) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] BroadcastHashJoin [wr_returned_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/explain.txt index 0078e83622..a036d5e090 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/explain.txt @@ -277,7 +277,7 @@ Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometFilter (51) @@ -350,7 +350,7 @@ Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (63) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt index 924b12cf01..61e40ee179 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/simplified.txt index d016a5ee2d..f54f284f87 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/simplified.txt @@ -17,21 +17,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #5 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt index 0078e83622..a036d5e090 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt @@ -277,7 +277,7 @@ Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometFilter (51) @@ -350,7 +350,7 @@ Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (63) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/extended.txt index 924b12cf01..61e40ee179 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter 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 d016a5ee2d..f54f284f87 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 @@ -17,21 +17,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #5 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/explain.txt index 31bc59394a..79fceba535 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (33) +CometNativeColumnarToRow (33) +- CometTakeOrderedAndProject (32) +- CometProject (31) +- CometBroadcastHashJoin (30) @@ -185,6 +185,6 @@ Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 Input [3]: [customer_id#20, customername#21, c_customer_id#7] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [customer_id#20, customername#21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/extended.txt index b025c868d1..402322f69f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/simplified.txt index 9df82b0e98..73465fcf59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/simplified.txt @@ -1,35 +1,33 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 - CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] - CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] - CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] +CometNativeColumnarToRow + CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] + CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] + CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastExchange [ca_address_sk] #2 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_city] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [cd_demo_sk] #3 + CometFilter [cd_demo_sk] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + CometBroadcastExchange [ib_income_band_sk] #5 + CometProject [ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + CometProject [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/explain.txt index ea154af6c8..3b2eefeaae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (52) +- * HashAggregate (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- CometColumnarExchange (49) +- * HashAggregate (48) +- * Project (47) @@ -22,32 +22,32 @@ TakeOrderedAndProject (52) : : : : : : : +- * Filter (3) : : : : : : : +- * ColumnarToRow (2) : : : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : : : +- * CometColumnarToRow (8) + : : : : : : +- CometNativeColumnarToRow (8) : : : : : : +- CometProject (7) : : : : : : +- CometFilter (6) : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (5) : : : : : +- BroadcastExchange (14) - : : : : : +- * CometColumnarToRow (13) + : : : : : +- CometNativeColumnarToRow (13) : : : : : +- CometFilter (12) : : : : : +- CometNativeScan parquet spark_catalog.default.web_page (11) : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) + : : : : +- CometNativeColumnarToRow (20) : : : : +- CometProject (19) : : : : +- CometFilter (18) : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (17) : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) + : : : +- CometNativeColumnarToRow (27) : : : +- CometProject (26) : : : +- CometFilter (25) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (24) : : +- BroadcastExchange (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometProject (33) : : +- CometFilter (32) : : +- CometNativeScan parquet spark_catalog.default.customer_address (31) : +- ReusedExchange (38) +- BroadcastExchange (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.reason (41) @@ -87,16 +87,16 @@ Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -(8) CometColumnarToRow +(8) CometNativeColumnarToRow Input [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -(9) BroadcastHashJoin [codegen id : 8] +(9) BroadcastHashJoin [codegen id : 2] Left keys [2]: [ws_item_sk#1, ws_order_number#3] Right keys [2]: [wr_item_sk#9, wr_order_number#14] Join type: Inner Join condition: None -(10) Project [codegen id : 8] +(10) Project [codegen id : 2] Output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] @@ -111,20 +111,20 @@ ReadSchema: struct Input [1]: [wp_web_page_sk#18] Condition : isnotnull(wp_web_page_sk#18) -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [1]: [wp_web_page_sk#18] (14) BroadcastExchange Input [1]: [wp_web_page_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 8] +(15) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_web_page_sk#2] Right keys [1]: [wp_web_page_sk#18] Join type: Inner Join condition: None -(16) Project [codegen id : 8] +(16) Project [codegen id : 2] Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] @@ -143,20 +143,20 @@ Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(static_invoke(CharVarcharC Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#23] -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] (21) BroadcastExchange Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 8] +(22) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_refunded_cdemo_sk#10] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) -(23) Project [codegen id : 8] +(23) Project [codegen id : 2] Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] @@ -175,20 +175,20 @@ Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(static_invoke(CharVarcharCo Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#26, 20)) AS cd_education_status#28] -(27) CometColumnarToRow [codegen id : 4] +(27) CometNativeColumnarToRow Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] (28) BroadcastExchange Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, string, true], input[2, string, true]),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 2] Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23] Right keys [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] Join type: Inner Join condition: None -(30) Project [codegen id : 8] +(30) Project [codegen id : 2] Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] @@ -207,33 +207,33 @@ Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) AS ca_state#32] -(34) CometColumnarToRow [codegen id : 5] +(34) CometNativeColumnarToRow Input [2]: [ca_address_sk#29, ca_state#32] (35) BroadcastExchange Input [2]: [ca_address_sk#29, ca_state#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_refunded_addr_sk#11] Right keys [1]: [ca_address_sk#29] Join type: Inner Join condition: ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) -(37) Project [codegen id : 8] +(37) Project [codegen id : 2] Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] (38) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#33] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#7] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(40) Project [codegen id : 8] +(40) Project [codegen id : 2] Output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] @@ -252,24 +252,24 @@ Condition : isnotnull(r_reason_sk#34) Input [2]: [r_reason_sk#34, r_reason_desc#35] Arguments: [r_reason_sk#34, r_reason_desc#36], [r_reason_sk#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(r_reason_desc#35, 100)) AS r_reason_desc#36] -(44) CometColumnarToRow [codegen id : 7] +(44) CometNativeColumnarToRow Input [2]: [r_reason_sk#34, r_reason_desc#36] (45) BroadcastExchange Input [2]: [r_reason_sk#34, r_reason_desc#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 8] +(46) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_reason_sk#13] Right keys [1]: [r_reason_sk#34] Join type: Inner Join condition: None -(47) Project [codegen id : 8] +(47) Project [codegen id : 2] Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#34, r_reason_desc#36] -(48) HashAggregate [codegen id : 8] +(48) HashAggregate [codegen id : 2] Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] Keys [1]: [r_reason_desc#36] Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] @@ -280,10 +280,10 @@ Results [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, coun Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] Arguments: hashpartitioning(r_reason_desc#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) CometColumnarToRow [codegen id : 9] +(50) CometNativeColumnarToRow Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] -(51) HashAggregate [codegen id : 9] +(51) HashAggregate [codegen id : 3] Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] Keys [1]: [r_reason_desc#36] Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] @@ -298,7 +298,7 @@ Arguments: 100, [substr(r_reason_desc, 1, 20)#52 ASC NULLS FIRST, avg(ws_quantit Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#33, d_year#56] Arguments: [d_date_sk#33], [d_date_sk#33] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt index 7f8b90a86c..aed7b365be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -23,40 +23,40 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_page : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/simplified.txt index 68d46e608d..e16d15b6c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [r_reason_desc] #1 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] BroadcastHashJoin [wr_reason_sk,r_reason_sk] @@ -29,55 +29,43 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometNativeColumnarToRow + CometFilter [wp_web_page_sk] + CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter + CometNativeColumnarToRow CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #7 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + CometNativeColumnarToRow + CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] + CometFilter [r_reason_sk,r_reason_desc] + CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/explain.txt index dff4c2b171..24184315bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/explain.txt @@ -278,7 +278,7 @@ Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunde Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -299,7 +299,7 @@ Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#33, d_year#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt index e2383394d1..5c782b95f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/simplified.txt index dfcca32f03..1e7b26f922 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/simplified.txt @@ -24,12 +24,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt index dff4c2b171..24184315bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt @@ -278,7 +278,7 @@ Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunde Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -299,7 +299,7 @@ Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#33, d_year#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/extended.txt index e2383394d1..5c782b95f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 dfcca32f03..1e7b26f922 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 @@ -24,12 +24,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/explain.txt index 829abbaec4..7c16497f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (24) +- * Project (23) +- Window (22) - +- * CometColumnarToRow (21) + +- CometNativeColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometColumnarExchange (16) +- * HashAggregate (15) +- * Expand (14) @@ -19,7 +19,7 @@ TakeOrderedAndProject (24) : : +- Scan parquet spark_catalog.default.web_sales (1) : +- ReusedExchange (4) +- BroadcastExchange (11) - +- * CometColumnarToRow (10) + +- CometNativeColumnarToRow (10) +- CometProject (9) +- CometFilter (8) +- CometNativeScan parquet spark_catalog.default.item (7) @@ -33,23 +33,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ws_item_sk#1, ws_net_paid#2] Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] @@ -68,28 +68,28 @@ Condition : isnotnull(i_item_sk#6) Input [3]: [i_item_sk#6, i_class#7, i_category#8] Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#7, 50)) AS i_class#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#8, 50)) AS i_category#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#6, i_class#9, i_category#10] (11) BroadcastExchange Input [3]: [i_item_sk#6, i_class#9, i_category#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [ws_net_paid#2, i_category#10, i_class#9] Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] -(14) Expand [codegen id : 3] +(14) Expand [codegen id : 1] Input [3]: [ws_net_paid#2, i_category#10, i_class#9] Arguments: [[ws_net_paid#2, i_category#10, i_class#9, 0], [ws_net_paid#2, i_category#10, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 1] Input [4]: [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] @@ -100,10 +100,10 @@ Results [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] Arguments: hashpartitioning(i_category#11, i_class#12, spark_grouping_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] @@ -118,14 +118,14 @@ Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(21) CometColumnarToRow [codegen id : 5] +(21) CometNativeColumnarToRow Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] (22) Window Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] -(23) Project [codegen id : 6] +(23) Project [codegen id : 3] Output [5]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] Input [8]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] @@ -137,7 +137,7 @@ Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ Condition : (((isnotnull(d_month_seq#23) AND (d_month_seq#23 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#23] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt index 41081debd9..a60a3158a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -18,17 +18,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/simplified.txt index 0b24fe234a..be50cc4b5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/simplified.txt @@ -1,44 +1,38 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] + Expand [ws_net_paid,i_category,i_class] + Project [ws_net_paid,i_category,i_class] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt index 54520c592a..0c74705433 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt @@ -138,7 +138,7 @@ Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -159,7 +159,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt index dfc810b108..6444b727e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt index 29e2d72920..d1e99b5c9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt index 54520c592a..0c74705433 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt @@ -138,7 +138,7 @@ Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -159,7 +159,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt index dfc810b108..6444b727e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 29e2d72920..d1e99b5c9b 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 @@ -20,12 +20,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/explain.txt index 4742064612..b3a7f003f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == * HashAggregate (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometColumnarExchange (50) +- * HashAggregate (49) +- * Project (48) +- * BroadcastHashJoin LeftAnti BuildRight (47) :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (17) + : :- CometNativeColumnarToRow (17) : : +- CometHashAggregate (16) : : +- CometColumnarExchange (15) : : +- * HashAggregate (14) @@ -19,12 +19,12 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer (7) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometHashAggregate (29) : +- CometColumnarExchange (28) : +- * HashAggregate (27) @@ -38,7 +38,7 @@ : : +- ReusedExchange (21) : +- ReusedExchange (24) +- BroadcastExchange (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometHashAggregate (44) +- CometColumnarExchange (43) +- * HashAggregate (42) @@ -61,23 +61,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) (4) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#4, d_date#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#2] Right keys [1]: [d_date_sk#4] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, d_date#5] Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] @@ -96,24 +96,24 @@ Condition : isnotnull(c_customer_sk#6) Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#7, 20)) AS c_first_name#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#8, 30)) AS c_last_name#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] (11) BroadcastExchange Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [c_last_name#10, c_first_name#9, d_date#5] Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [c_last_name#10, c_first_name#9, d_date#5] Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] Functions: [] @@ -129,7 +129,7 @@ Input [3]: [c_last_name#10, c_first_name#9, d_date#5] Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] Functions: [] -(17) CometColumnarToRow [codegen id : 12] +(17) CometNativeColumnarToRow Input [3]: [c_last_name#10, c_first_name#9, d_date#5] (18) Scan parquet spark_catalog.default.catalog_sales @@ -140,40 +140,40 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -(20) Filter [codegen id : 6] +(20) Filter [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Condition : isnotnull(cs_bill_customer_sk#11) (21) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#13, d_date#14] -(22) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(23) Project [codegen id : 2] Output [2]: [cs_bill_customer_sk#11, d_date#14] Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] (24) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_bill_customer_sk#11] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(26) Project [codegen id : 2] Output [3]: [c_last_name#17, c_first_name#16, d_date#14] Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] -(27) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 2] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] @@ -189,14 +189,14 @@ Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (31) BroadcastExchange Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 12] +(32) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join type: LeftAnti @@ -210,40 +210,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(35) Filter [codegen id : 10] +(35) Filter [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Condition : isnotnull(ws_bill_customer_sk#18) (36) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#20, d_date#21] -(37) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(38) Project [codegen id : 3] Output [2]: [ws_bill_customer_sk#18, d_date#21] Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] (39) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_bill_customer_sk#18] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 3] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 3] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] @@ -259,24 +259,24 @@ Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] -(45) CometColumnarToRow [codegen id : 11] +(45) CometNativeColumnarToRow Input [3]: [c_last_name#24, c_first_name#23, d_date#21] (46) BroadcastExchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 12] +(47) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] Join type: LeftAnti Join condition: None -(48) Project [codegen id : 12] +(48) Project [codegen id : 4] Output: [] Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -(49) HashAggregate [codegen id : 12] +(49) HashAggregate [codegen id : 4] Input: [] Keys: [] Functions [1]: [partial_count(1)] @@ -287,10 +287,10 @@ Results [1]: [count#26] Input [1]: [count#26] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) CometColumnarToRow [codegen id : 13] +(51) CometNativeColumnarToRow Input [1]: [count#26] -(52) HashAggregate [codegen id : 13] +(52) HashAggregate [codegen id : 5] Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] @@ -301,7 +301,7 @@ Results [1]: [count(1)#27 AS count(1)#28] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -322,7 +322,7 @@ Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_mo Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt index 638b8865e0..7a9e62d57c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt @@ -1,11 +1,11 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -18,22 +18,22 @@ HashAggregate : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometColumnarExchange : +- HashAggregate @@ -46,17 +46,17 @@ HashAggregate : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate @@ -69,12 +69,12 @@ HashAggregate : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/simplified.txt index afc302e779..25e4104be0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (13) +WholeStageCodegen (5) HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (12) + WholeStageCodegen (4) HashAggregate [count,count] Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [c_last_name,c_first_name,d_date] CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] BroadcastHashJoin [ss_customer_sk,c_customer_sk] @@ -24,63 +24,55 @@ WholeStageCodegen (13) Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter ReusedExchange [d_date_sk,d_date] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometNativeColumnarToRow + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometNativeColumnarToRow + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/explain.txt index 3dd56d08f4..85308d1c4d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/explain.txt @@ -292,7 +292,7 @@ Results [1]: [count(1)#30 AS count(1)#31] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (56) -+- * CometColumnarToRow (55) ++- CometNativeColumnarToRow (55) +- CometProject (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -313,7 +313,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(55) CometColumnarToRow [codegen id : 1] +(55) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (56) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt index ddee139acf..0c51ae09a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ HashAggregate : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/simplified.txt index f687139735..e1555c4783 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt index 3dd56d08f4..85308d1c4d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt @@ -292,7 +292,7 @@ Results [1]: [count(1)#30 AS count(1)#31] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (56) -+- * CometColumnarToRow (55) ++- CometNativeColumnarToRow (55) +- CometProject (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -313,7 +313,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(55) CometColumnarToRow [codegen id : 1] +(55) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (56) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/extended.txt index ddee139acf..0c51ae09a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/extended.txt @@ -17,7 +17,7 @@ HashAggregate : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt index f687139735..e1555c4783 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/explain.txt index 25b4c305f2..d2d4ce18d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/explain.txt @@ -6,7 +6,7 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) +: : : : : : :- CometNativeColumnarToRow (25) : : : : : : : +- CometHashAggregate (24) : : : : : : : +- CometExchange (23) : : : : : : : +- CometHashAggregate (22) @@ -32,7 +32,7 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) +: : : : : : +- CometNativeColumnarToRow (44) : : : : : : +- CometHashAggregate (43) : : : : : : +- CometExchange (42) : : : : : : +- CometHashAggregate (41) @@ -52,7 +52,7 @@ : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) +: : : : : +- CometNativeColumnarToRow (65) : : : : : +- CometHashAggregate (64) : : : : : +- CometExchange (63) : : : : : +- CometHashAggregate (62) @@ -72,7 +72,7 @@ : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) +: : : : +- CometNativeColumnarToRow (86) : : : : +- CometHashAggregate (85) : : : : +- CometExchange (84) : : : : +- CometHashAggregate (83) @@ -92,7 +92,7 @@ : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) +: : : +- CometNativeColumnarToRow (107) : : : +- CometHashAggregate (106) : : : +- CometExchange (105) : : : +- CometHashAggregate (104) @@ -112,7 +112,7 @@ : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) +: : +- CometNativeColumnarToRow (128) : : +- CometHashAggregate (127) : : +- CometExchange (126) : : +- CometHashAggregate (125) @@ -132,7 +132,7 @@ : : : +- CometNativeScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) +: +- CometNativeColumnarToRow (149) : +- CometHashAggregate (148) : +- CometExchange (147) : +- CometHashAggregate (146) @@ -152,7 +152,7 @@ : : +- CometNativeScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * CometColumnarToRow (170) + +- CometNativeColumnarToRow (170) +- CometHashAggregate (169) +- CometExchange (168) +- CometHashAggregate (167) @@ -285,7 +285,7 @@ Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -(25) CometColumnarToRow [codegen id : 8] +(25) CometNativeColumnarToRow Input [1]: [h8_30_to_9#14] (26) CometNativeScan parquet spark_catalog.default.store_sales @@ -368,14 +368,14 @@ Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [h9_to_9_30#25] (45) BroadcastExchange Input [1]: [h9_to_9_30#25] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 8] +(46) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -459,14 +459,14 @@ Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -(65) CometColumnarToRow [codegen id : 2] +(65) CometNativeColumnarToRow Input [1]: [h9_30_to_10#36] (66) BroadcastExchange Input [1]: [h9_30_to_10#36] Arguments: IdentityBroadcastMode, [plan_id=5] -(67) BroadcastNestedLoopJoin [codegen id : 8] +(67) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -550,14 +550,14 @@ Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -(86) CometColumnarToRow [codegen id : 3] +(86) CometNativeColumnarToRow Input [1]: [h10_to_10_30#47] (87) BroadcastExchange Input [1]: [h10_to_10_30#47] Arguments: IdentityBroadcastMode, [plan_id=7] -(88) BroadcastNestedLoopJoin [codegen id : 8] +(88) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -641,14 +641,14 @@ Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -(107) CometColumnarToRow [codegen id : 4] +(107) CometNativeColumnarToRow Input [1]: [h10_30_to_11#58] (108) BroadcastExchange Input [1]: [h10_30_to_11#58] Arguments: IdentityBroadcastMode, [plan_id=9] -(109) BroadcastNestedLoopJoin [codegen id : 8] +(109) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -732,14 +732,14 @@ Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -(128) CometColumnarToRow [codegen id : 5] +(128) CometNativeColumnarToRow Input [1]: [h11_to_11_30#69] (129) BroadcastExchange Input [1]: [h11_to_11_30#69] Arguments: IdentityBroadcastMode, [plan_id=11] -(130) BroadcastNestedLoopJoin [codegen id : 8] +(130) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -823,14 +823,14 @@ Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -(149) CometColumnarToRow [codegen id : 6] +(149) CometNativeColumnarToRow Input [1]: [h11_30_to_12#80] (150) BroadcastExchange Input [1]: [h11_30_to_12#80] Arguments: IdentityBroadcastMode, [plan_id=13] -(151) BroadcastNestedLoopJoin [codegen id : 8] +(151) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -914,14 +914,14 @@ Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -(170) CometColumnarToRow [codegen id : 7] +(170) CometNativeColumnarToRow Input [1]: [h12_to_12_30#91] (171) BroadcastExchange Input [1]: [h12_to_12_30#91] Arguments: IdentityBroadcastMode, [plan_id=15] -(172) BroadcastNestedLoopJoin [codegen id : 8] +(172) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/extended.txt index 24f596de9b..87f19deeb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/extended.txt @@ -5,7 +5,7 @@ BroadcastNestedLoopJoin : : : :- BroadcastNestedLoopJoin : : : : :- BroadcastNestedLoopJoin : : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : : : :- CometColumnarToRow +: : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometHashAggregate : : : : : : : +- CometExchange : : : : : : : +- CometHashAggregate @@ -31,7 +31,7 @@ BroadcastNestedLoopJoin : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : : : +- BroadcastExchange -: : : : : : +- CometColumnarToRow +: : : : : : +- CometNativeColumnarToRow : : : : : : +- CometHashAggregate : : : : : : +- CometExchange : : : : : : +- CometHashAggregate @@ -57,7 +57,7 @@ BroadcastNestedLoopJoin : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : : +- BroadcastExchange -: : : : : +- CometColumnarToRow +: : : : : +- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometExchange : : : : : +- CometHashAggregate @@ -83,7 +83,7 @@ BroadcastNestedLoopJoin : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : +- BroadcastExchange -: : : : +- CometColumnarToRow +: : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -109,7 +109,7 @@ BroadcastNestedLoopJoin : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange -: : : +- CometColumnarToRow +: : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -135,7 +135,7 @@ BroadcastNestedLoopJoin : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -161,7 +161,7 @@ BroadcastNestedLoopJoin : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -187,7 +187,7 @@ BroadcastNestedLoopJoin : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/simplified.txt index cef8e0e760..82809b3066 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (8) +WholeStageCodegen (1) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin @@ -6,8 +6,8 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [count] [h8_30_to_9,count(1)] CometExchange #1 CometHashAggregate [count] @@ -34,57 +34,9 @@ WholeStageCodegen (8) CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 + CometNativeColumnarToRow + CometHashAggregate [count] [h9_to_9_30,count(1)] + CometExchange #6 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -96,18 +48,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #13 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 + InputAdapter + BroadcastExchange #8 + CometNativeColumnarToRow + CometHashAggregate [count] [h9_30_to_10,count(1)] + CometExchange #9 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -119,18 +69,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #16 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 + InputAdapter + BroadcastExchange #11 + CometNativeColumnarToRow + CometHashAggregate [count] [h10_to_10_30,count(1)] + CometExchange #12 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -142,18 +90,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #19 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 + InputAdapter + BroadcastExchange #14 + CometNativeColumnarToRow + CometHashAggregate [count] [h10_30_to_11,count(1)] + CometExchange #15 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -165,18 +111,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #22 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 + InputAdapter + BroadcastExchange #17 + CometNativeColumnarToRow + CometHashAggregate [count] [h11_to_11_30,count(1)] + CometExchange #18 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -188,8 +132,50 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #25 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #20 + CometNativeColumnarToRow + CometHashAggregate [count] [h11_30_to_12,count(1)] + CometExchange #21 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #22 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #23 + CometNativeColumnarToRow + CometHashAggregate [count] [h12_to_12_30,count(1)] + CometExchange #24 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #25 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/explain.txt index 54bc4a6948..c5047ab02e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (30) +- * Project (29) +- * Filter (28) +- Window (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -16,7 +16,7 @@ TakeOrderedAndProject (30) : +- * BroadcastHashJoin Inner BuildRight (12) : :- * Project (10) : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject (30) : : +- Scan parquet spark_catalog.default.store_sales (5) : +- ReusedExchange (11) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -46,7 +46,7 @@ Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category# Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) AS i_class#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) AS i_category#7] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] (5) Scan parquet spark_catalog.default.store_sales @@ -68,26 +68,26 @@ Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] (11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#13, d_moy#14] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#11] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14] Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#14] @@ -102,24 +102,24 @@ ReadSchema: struct Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] Condition : isnotnull(s_store_sk#15) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] (17) BroadcastExchange Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#9] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14, s_store_sk#15, s_store_name#16, s_company_name#17] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] @@ -130,10 +130,10 @@ Results [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_nam Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] @@ -148,18 +148,18 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#16, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#16 ASC NULLS FIRST, s_company_name#17 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] (27) Window Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#16, s_company_name#17, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#16, s_company_name#17] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END -(29) Project [codegen id : 7] +(29) Project [codegen id : 4] Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] @@ -171,7 +171,7 @@ Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -192,7 +192,7 @@ Condition : ((isnotnull(d_year#24) AND (d_year#24 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#13, d_year#24, d_moy#14] Arguments: [d_date_sk#13, d_moy#14], [d_date_sk#13, d_moy#14] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_moy#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt index 0b6c2edaa0..5c7a26057d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject +- Project +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -25,17 +25,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/simplified.txt index f87ef33db4..573b3eeaf0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/simplified.txt @@ -1,53 +1,47 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk,s_store_name,s_company_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt index 185c9d264c..6366035e88 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_moy#15] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt index 61bfd1d960..e64a4daac6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt index 185c9d264c..6366035e88 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_moy#15] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 61bfd1d960..e64a4daac6 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/explain.txt index da194f2825..927b31b896 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * CometColumnarToRow (3) ++- CometNativeColumnarToRow (3) +- CometFilter (2) +- CometNativeScan parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) CometColumnarToRow [codegen id : 1] +(3) CometNativeColumnarToRow Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] @@ -26,7 +26,7 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) +CometNativeColumnarToRow (12) +- CometProject (11) +- CometHashAggregate (10) +- CometExchange (9) @@ -69,7 +69,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(Unscal Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] -(12) CometColumnarToRow [codegen id : 1] +(12) CometNativeColumnarToRow Input [1]: [mergedValue#29] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] @@ -77,7 +77,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) +CometNativeColumnarToRow (20) +- CometProject (19) +- CometHashAggregate (18) +- CometExchange (17) @@ -120,7 +120,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(Unscal Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] -(20) CometColumnarToRow [codegen id : 1] +(20) CometNativeColumnarToRow Input [1]: [mergedValue#42] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] @@ -128,7 +128,7 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) +CometNativeColumnarToRow (28) +- CometProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -171,7 +171,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(Unscal Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [mergedValue#55] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] @@ -179,7 +179,7 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) +CometNativeColumnarToRow (36) +- CometProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -222,7 +222,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(Unscal Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [mergedValue#68] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] @@ -230,7 +230,7 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) +CometNativeColumnarToRow (44) +- CometProject (43) +- CometHashAggregate (42) +- CometExchange (41) @@ -273,7 +273,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(Unscal Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [mergedValue#81] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/extended.txt index 53ba3252cf..ff39852ed0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/extended.txt @@ -1,6 +1,6 @@ Project [COMET: ] : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -11,7 +11,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -22,7 +22,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -33,7 +33,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -44,7 +44,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -54,7 +54,7 @@ : : +- CometNativeScan parquet spark_catalog.default.store_sales : :- ReusedSubquery : +- ReusedSubquery -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/simplified.txt index 9593d6ebd1..813cb91b2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/simplified.txt @@ -1,71 +1,61 @@ WholeStageCodegen (1) Project Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [r_reason_sk] CometNativeScan parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/explain.txt index b3f32555de..14dc1ef68d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * CometColumnarToRow (3) ++- CometNativeColumnarToRow (3) +- CometFilter (2) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) CometColumnarToRow [codegen id : 1] +(3) CometNativeColumnarToRow Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt index ed71033b26..a86d85c820 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/extended.txt @@ -54,7 +54,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : :- ReusedSubquery : +- ReusedSubquery -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/simplified.txt index 558f5f4b36..9b54130ef1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/simplified.txt @@ -65,7 +65,7 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [r_reason_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt index b3f32555de..14dc1ef68d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * CometColumnarToRow (3) ++- CometNativeColumnarToRow (3) +- CometFilter (2) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) CometColumnarToRow [codegen id : 1] +(3) CometNativeColumnarToRow Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/extended.txt index ed71033b26..a86d85c820 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/extended.txt @@ -54,7 +54,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : :- ReusedSubquery : +- ReusedSubquery -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason 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 558f5f4b36..9b54130ef1 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 @@ -65,7 +65,7 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [r_reason_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/explain.txt index 73b54f439b..71001884ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) + :- CometNativeColumnarToRow (25) : +- CometHashAggregate (24) : +- CometExchange (23) : +- CometHashAggregate (22) @@ -27,7 +27,7 @@ : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometHashAggregate (43) +- CometExchange (42) +- CometHashAggregate (41) @@ -160,7 +160,7 @@ Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -(25) CometColumnarToRow [codegen id : 2] +(25) CometNativeColumnarToRow Input [1]: [amc#12] (26) CometNativeScan parquet spark_catalog.default.web_sales @@ -243,18 +243,18 @@ Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [pmc#22] (45) BroadcastExchange Input [1]: [pmc#22] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 2] +(46) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(47) Project [codegen id : 2] +(47) Project [codegen id : 1] Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] Input [2]: [amc#12, pmc#22] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/extended.txt index 977fcbbc6b..b1125e15e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/extended.txt @@ -1,6 +1,6 @@ Project +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -26,7 +26,7 @@ Project : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/simplified.txt index a13072cd91..686e05404c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (2) +WholeStageCodegen (1) Project [amc,pmc] BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [count] [amc,count(1)] CometExchange #1 CometHashAggregate [count] @@ -29,24 +29,22 @@ WholeStageCodegen (2) CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - ReusedExchange [wp_web_page_sk] #4 + CometNativeColumnarToRow + CometHashAggregate [count] [pmc,count(1)] + CometExchange #6 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #7 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/explain.txt index f6cfa7bdef..520947f724 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -18,7 +18,7 @@ : : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : : :- * Project (10) : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * CometColumnarToRow (4) + : : : : : :- CometNativeColumnarToRow (4) : : : : : : +- CometProject (3) : : : : : : +- CometFilter (2) : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center (1) @@ -28,21 +28,21 @@ : : : : : +- Scan parquet spark_catalog.default.catalog_returns (5) : : : : +- ReusedExchange (11) : : : +- BroadcastExchange (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometFilter (15) : : : +- CometNativeScan parquet spark_catalog.default.customer (14) : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometProject (22) : : +- CometFilter (21) : : +- CometNativeScan parquet spark_catalog.default.customer_address (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_demographics (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.household_demographics (34) @@ -63,7 +63,7 @@ Condition : isnotnull(cc_call_center_sk#1) Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(cc_call_center_id#2, 16)) AS cc_call_center_id#5, cc_name#3, cc_manager#4] -(4) CometColumnarToRow [codegen id : 7] +(4) CometNativeColumnarToRow Input [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] (5) Scan parquet spark_catalog.default.catalog_returns @@ -85,26 +85,26 @@ Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_ Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cc_call_center_sk#1] Right keys [1]: [cr_call_center_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 2] Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] (11) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#11] -(12) BroadcastHashJoin [codegen id : 7] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returned_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 7] +(13) Project [codegen id : 2] Output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] @@ -119,20 +119,20 @@ ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 3] Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) @@ -62,20 +62,20 @@ Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotn Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [i_item_sk#5] (8) BroadcastExchange Input [1]: [i_item_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 6] +(9) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 6] +(10) Project [codegen id : 3] Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] @@ -87,27 +87,27 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] +(12) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -(13) Filter [codegen id : 3] +(13) Filter [codegen id : 1] Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] Condition : isnotnull(ws_item_sk#7) (14) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#10] -(15) BroadcastHashJoin [codegen id : 3] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(16) Project [codegen id : 3] +(16) Project [codegen id : 1] Output [2]: [ws_item_sk#7, ws_ext_discount_amt#8] Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#10] -(17) HashAggregate [codegen id : 3] +(17) HashAggregate [codegen id : 1] Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] Keys [1]: [ws_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] @@ -118,17 +118,17 @@ Results [3]: [ws_item_sk#7, sum#13, count#14] Input [3]: [ws_item_sk#7, sum#13, count#14] Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) CometColumnarToRow [codegen id : 4] +(19) CometNativeColumnarToRow Input [3]: [ws_item_sk#7, sum#13, count#14] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#7, sum#13, count#14] Keys [1]: [ws_item_sk#7] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#15] Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -(21) Filter [codegen id : 4] +(21) Filter [codegen id : 2] Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) @@ -136,30 +136,30 @@ Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#5] Right keys [1]: [ws_item_sk#7] Join type: Inner Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#16) -(24) Project [codegen id : 6] +(24) Project [codegen id : 3] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] (25) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#17] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 3] Output [1]: [ws_ext_discount_amt#2] Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#17] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 3] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] @@ -170,10 +170,10 @@ Results [1]: [sum#19] Input [1]: [sum#19] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [1]: [sum#19] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 4] Input [1]: [sum#19] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] @@ -184,7 +184,7 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#20,17,2) AS Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -205,7 +205,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#2 Input [2]: [d_date_sk#17, d_date#22] Arguments: [d_date_sk#17], [d_date_sk#17] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt index 2d15266049..b3ca85cdaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -13,19 +13,19 @@ HashAggregate : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -35,12 +35,12 @@ HashAggregate : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/simplified.txt index 17df728966..71aeded80b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [ws_ext_discount_amt] [sum,sum] Project [ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -17,29 +17,25 @@ WholeStageCodegen (7) Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(ws_ext_discount_amt))] HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ws_item_sk] #5 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] Project [ws_item_sk,ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt index cdc8dbc3bc..1b2f594125 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt @@ -175,7 +175,7 @@ Input [1]: [Excess Discount Amount #18] Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt index 20df9a8047..ca50f78e18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt index 4ec5755df7..991100027b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt index cdc8dbc3bc..1b2f594125 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt @@ -175,7 +175,7 @@ Input [1]: [Excess Discount Amount #18] Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt index 20df9a8047..ca50f78e18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 4ec5755df7..991100027b 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 @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/explain.txt index 4b430e436e..b054fe194a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (22) +CometNativeColumnarToRow (22) +- CometTakeOrderedAndProject (21) +- CometHashAggregate (20) +- CometExchange (19) @@ -119,6 +119,6 @@ Functions [1]: [sum(act_sales#14)] Input [2]: [ss_customer_sk#2, sumsales#17] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] -(22) CometColumnarToRow [codegen id : 1] +(22) CometNativeColumnarToRow Input [2]: [ss_customer_sk#2, sumsales#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/extended.txt index 7203a678b0..1496f26233 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/simplified.txt index c0b74010df..f08009ad54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/simplified.txt @@ -1,24 +1,22 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - 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] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] +CometNativeColumnarToRow + CometTakeOrderedAndProject [ss_customer_sk,sumsales] + CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] + CometExchange [ss_customer_sk] #1 + 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] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometExchange [ss_item_sk,ss_ticket_number] #2 + CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometExchange [sr_item_sk,sr_ticket_number] #3 + CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometBroadcastExchange [r_reason_sk] #4 + CometProject [r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] + CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/explain.txt index 2f78d0dc4b..b3be563893 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (41) +CometNativeColumnarToRow (41) +- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -204,7 +204,7 @@ Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [3]: [ws_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -230,6 +230,6 @@ Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -(41) CometColumnarToRow [codegen id : 2] +(41) CometNativeColumnarToRow Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/extended.txt index e5c7b8688a..c91f0410ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/extended.txt @@ -1,9 +1,9 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/simplified.txt index feab73bcd6..cd992a9669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/simplified.txt @@ -1,45 +1,43 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_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] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_warehouse_sk] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] +CometNativeColumnarToRow + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + 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] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_warehouse_sk] + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometExchange [ws_order_number] #2 + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometSort [ws_warehouse_sk,ws_order_number] + CometExchange [ws_order_number] #3 + CometProject [ws_warehouse_sk,ws_order_number] + CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometSort [wr_order_number] + CometExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/explain.txt index a65b77bf6a..79b466250a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (54) +CometNativeColumnarToRow (54) +- CometHashAggregate (53) +- CometColumnarExchange (52) +- * HashAggregate (51) +- * HashAggregate (50) - +- * CometColumnarToRow (49) + +- CometNativeColumnarToRow (49) +- CometHashAggregate (48) +- CometProject (47) +- CometBroadcastHashJoin (46) @@ -271,7 +271,7 @@ Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [3]: [ws_order_number#4, sum#25, sum#26] (50) HashAggregate [codegen id : 1] @@ -297,6 +297,6 @@ Input [3]: [sum#25, sum#26, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -(54) CometColumnarToRow [codegen id : 2] +(54) CometNativeColumnarToRow Input [3]: [order count #31, total shipping cost #32, total net profit #33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/extended.txt index cbba03d28d..94e8b8d087 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/extended.txt @@ -1,9 +1,9 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/simplified.txt index 15f29c507c..b81cc72852 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/simplified.txt @@ -1,58 +1,56 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_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] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] +CometNativeColumnarToRow + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + 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] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometExchange [ws_order_number] #2 + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometProject [ws_order_number] + CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] + CometSort [ws_warehouse_sk,ws_order_number] + CometExchange [ws_order_number] #3 + CometProject [ws_warehouse_sk,ws_order_number] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometProject [wr_order_number] + CometSortMergeJoin [wr_order_number,ws_order_number] + CometSort [wr_order_number] + CometExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + CometProject [ws_order_number] + CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/explain.txt index db701aae34..7ca69ec813 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (25) +CometNativeColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -138,6 +138,6 @@ Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -(25) CometColumnarToRow [codegen id : 1] +(25) CometNativeColumnarToRow Input [1]: [count(1)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/extended.txt index 170d3b7879..9ee165830a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/simplified.txt index b40f41659f..6599ccccb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/simplified.txt @@ -1,27 +1,25 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] +CometNativeColumnarToRow + CometHashAggregate [count] [count(1),count(1)] + CometExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/explain.txt index a7f1cb451d..0121fd183f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (24) +CometNativeColumnarToRow (24) +- CometHashAggregate (23) +- CometExchange (22) +- CometHashAggregate (21) @@ -32,23 +32,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] (3) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] -(4) BroadcastHashJoin [codegen id : 2] +(4) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(5) Project [codegen id : 2] +(5) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_customer_sk#2] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] -(6) HashAggregate [codegen id : 2] +(6) HashAggregate [codegen id : 1] Input [2]: [ss_item_sk#1, ss_customer_sk#2] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] @@ -75,23 +75,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 2] Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] (12) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#11] -(13) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#10] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(14) Project [codegen id : 4] +(14) Project [codegen id : 2] Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#11] -(15) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 2] Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Functions: [] @@ -134,14 +134,14 @@ Input [3]: [sum#14, sum#15, sum#16] Keys: [] Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END)] -(24) CometColumnarToRow [codegen id : 5] +(24) CometNativeColumnarToRow Input [3]: [store_only#17, catalog_only#18, store_and_catalog#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -162,7 +162,7 @@ Condition : (((isnotnull(d_month_seq#20) AND (d_month_seq#20 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#20] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt index 798bfdaf1b..e3b777a05a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -14,12 +14,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -33,7 +33,7 @@ CometColumnarToRow : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/simplified.txt index dac440b122..04cb3e457d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/simplified.txt @@ -1,41 +1,37 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - 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 [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] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (4) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 +CometNativeColumnarToRow + 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 [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] + CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] + CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_customer_sk,ss_item_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + CometSort [customer_sk,item_sk] + CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] + CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 + WholeStageCodegen (2) + HashAggregate [cs_bill_customer_sk,cs_item_sk] + Project [cs_bill_customer_sk,cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/explain.txt index 420f37fccc..a21fa2b802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/explain.txt @@ -146,7 +146,7 @@ Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt index e1aefb8382..0969a0e796 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/simplified.txt index a7de047b25..d37700ec0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/simplified.txt @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt index 420f37fccc..a21fa2b802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt @@ -146,7 +146,7 @@ Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/extended.txt index e1aefb8382..0969a0e796 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a7de047b25..d37700ec0f 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 @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/explain.txt index 32c4235d27..820373caae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -* CometColumnarToRow (26) +CometNativeColumnarToRow (26) +- CometProject (25) +- CometSort (24) +- CometColumnarExchange (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -20,7 +20,7 @@ : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.store_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -35,10 +35,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) @@ -57,37 +57,37 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10 Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -98,10 +98,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -116,14 +116,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] (21) Window Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] @@ -139,14 +139,14 @@ Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrev Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(26) CometColumnarToRow [codegen id : 7] +(26) CometNativeColumnarToRow Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.date_dim (27) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt index 030031856f..af8cf29205 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt @@ -1,14 +1,14 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometProject +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -20,17 +20,17 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/simplified.txt index 3f64b57ffa..91f3f3c423 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/simplified.txt @@ -1,48 +1,40 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] +CometNativeColumnarToRow + CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + CometNativeColumnarToRow + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt index 158e7520c2..d02e50615f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt @@ -147,7 +147,7 @@ Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrev Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt index 4c972848e7..152665febf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt index af05a7d8c6..8c213757be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt @@ -24,12 +24,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt index 158e7520c2..d02e50615f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt @@ -147,7 +147,7 @@ Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrev Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt index 4c972848e7..152665febf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 af05a7d8c6..8c213757be 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 @@ -24,12 +24,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/explain.txt index 12e87af3b5..c483d048ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (29) +CometNativeColumnarToRow (29) +- CometTakeOrderedAndProject (28) +- CometHashAggregate (27) +- CometExchange (26) @@ -163,6 +163,6 @@ Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt index c075163849..c236a1647f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/simplified.txt index 98a215dbc6..4c6dcfd6fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/simplified.txt @@ -1,31 +1,29 @@ -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 [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 [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] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +CometNativeColumnarToRow + 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 [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 [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] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometProject [sm_type] [sm_ship_mode_sk,sm_type] + CometFilter [sm_ship_mode_sk,sm_type] + CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/explain.txt index 5943563361..574eb40617 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (44) : : +- * BroadcastHashJoin Inner BuildRight (28) : : :- * Filter (11) : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometColumnarExchange (8) : : : +- * HashAggregate (7) : : : +- * Project (6) @@ -20,11 +20,11 @@ TakeOrderedAndProject (44) : : +- BroadcastExchange (27) : : +- * Filter (26) : : +- * HashAggregate (25) - : : +- * CometColumnarToRow (24) + : : +- CometNativeColumnarToRow (24) : : +- CometColumnarExchange (23) : : +- * HashAggregate (22) : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (20) + : : +- CometNativeColumnarToRow (20) : : +- CometColumnarExchange (19) : : +- * HashAggregate (18) : : +- * Project (17) @@ -34,12 +34,12 @@ TakeOrderedAndProject (44) : : : +- Scan parquet spark_catalog.default.store_returns (12) : : +- ReusedExchange (15) : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- CometProject (32) : +- CometFilter (31) : +- CometNativeScan parquet spark_catalog.default.store (30) +- BroadcastExchange (41) - +- * CometColumnarToRow (40) + +- CometNativeColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometNativeScan parquet spark_catalog.default.customer (37) @@ -53,27 +53,27 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 2] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [sr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 2] +(6) Project [codegen id : 1] Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -(7) HashAggregate [codegen id : 2] +(7) HashAggregate [codegen id : 1] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] @@ -84,17 +84,17 @@ Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 9] +(9) CometNativeColumnarToRow Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -(10) HashAggregate [codegen id : 9] +(10) HashAggregate [codegen id : 5] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(11) Filter [codegen id : 9] +(11) Filter [codegen id : 5] Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] Condition : isnotnull(ctr_total_return#12) @@ -106,27 +106,27 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] +(13) ColumnarToRow [codegen id : 2] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -(14) Filter [codegen id : 4] +(14) Filter [codegen id : 2] Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] Condition : isnotnull(sr_store_sk#2) (15) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#6] -(16) BroadcastHashJoin [codegen id : 4] +(16) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(17) Project [codegen id : 4] +(17) Project [codegen id : 2] Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] @@ -137,17 +137,17 @@ Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] -(21) HashAggregate [codegen id : 5] +(21) HashAggregate [codegen id : 3] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] Results [2]: [sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(22) HashAggregate [codegen id : 5] +(22) HashAggregate [codegen id : 3] Input [2]: [ctr_store_sk#11, ctr_total_return#12] Keys [1]: [ctr_store_sk#11] Functions [1]: [partial_avg(ctr_total_return#12)] @@ -158,17 +158,17 @@ Results [3]: [ctr_store_sk#11, sum#17, count#18] Input [3]: [ctr_store_sk#11, sum#17, count#18] Arguments: hashpartitioning(ctr_store_sk#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 6] +(24) CometNativeColumnarToRow Input [3]: [ctr_store_sk#11, sum#17, count#18] -(25) HashAggregate [codegen id : 6] +(25) HashAggregate [codegen id : 4] Input [3]: [ctr_store_sk#11, sum#17, count#18] Keys [1]: [ctr_store_sk#11] Functions [1]: [avg(ctr_total_return#12)] Aggregate Attributes [1]: [avg(ctr_total_return#12)#19] Results [2]: [(avg(ctr_total_return#12)#19 * 1.2) AS (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11 AS ctr_store_sk#11#21] -(26) Filter [codegen id : 6] +(26) Filter [codegen id : 4] Input [2]: [(avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21] Condition : isnotnull((avg(ctr_total_return) * 1.2)#20) @@ -176,13 +176,13 @@ Condition : isnotnull((avg(ctr_total_return) * 1.2)#20) Input [2]: [(avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] -(28) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_store_sk#11] Right keys [1]: [ctr_store_sk#11#21] Join type: Inner Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#20) -(29) Project [codegen id : 9] +(29) Project [codegen id : 5] Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21] @@ -201,20 +201,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#22, s_state#23] Arguments: [s_store_sk#22], [s_store_sk#22] -(33) CometColumnarToRow [codegen id : 7] +(33) CometNativeColumnarToRow Input [1]: [s_store_sk#22] (34) BroadcastExchange Input [1]: [s_store_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(35) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_store_sk#11] Right keys [1]: [s_store_sk#22] Join type: Inner Join condition: None -(36) Project [codegen id : 9] +(36) Project [codegen id : 5] Output [1]: [ctr_customer_sk#10] Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#22] @@ -233,20 +233,20 @@ Condition : isnotnull(c_customer_sk#24) Input [2]: [c_customer_sk#24, c_customer_id#25] Arguments: [c_customer_sk#24, c_customer_id#26], [c_customer_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#25, 16, true, false, true) AS c_customer_id#26] -(40) CometColumnarToRow [codegen id : 8] +(40) CometNativeColumnarToRow Input [2]: [c_customer_sk#24, c_customer_id#26] (41) BroadcastExchange Input [2]: [c_customer_sk#24, c_customer_id#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(42) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_customer_sk#10] Right keys [1]: [c_customer_sk#24] Join type: Inner Join condition: None -(43) Project [codegen id : 9] +(43) Project [codegen id : 5] Output [1]: [c_customer_id#26] Input [3]: [ctr_customer_sk#10, c_customer_sk#24, c_customer_id#26] @@ -258,7 +258,7 @@ Arguments: 100, [c_customer_id#26 ASC NULLS FIRST], [c_customer_id#26] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometNativeScan parquet spark_catalog.default.date_dim (45) @@ -279,7 +279,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#6, d_year#27] Arguments: [d_date_sk#6], [d_date_sk#6] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt index 0622aad9d5..91b07a7538 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -17,23 +17,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -43,17 +43,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/simplified.txt index b45596c7a2..63536ff0e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (9) + WholeStageCodegen (5) Project [c_customer_id] BroadcastHashJoin [ctr_customer_sk,c_customer_sk] Project [ctr_customer_sk] @@ -8,10 +8,10 @@ TakeOrderedAndProject [c_customer_id] BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] Project [sr_customer_sk,sr_store_sk,sr_return_amt] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -21,29 +21,27 @@ TakeOrderedAndProject [c_customer_id] Scan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (6) + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ctr_store_sk] #4 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] Project [sr_customer_sk,sr_store_sk,sr_return_amt] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -56,17 +54,13 @@ TakeOrderedAndProject [c_customer_id] ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] + CometNativeColumnarToRow + CometProject [c_customer_id] [c_customer_sk,c_customer_id] + CometFilter [c_customer_sk,c_customer_id] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/explain.txt index b8ba9b884c..dbb92bf091 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/explain.txt @@ -236,7 +236,7 @@ Input [1]: [c_customer_id#22] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt index 8aa14c43dd..0d0d80c7f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/simplified.txt index c745a5b255..4900411443 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index b8ba9b884c..dbb92bf091 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -236,7 +236,7 @@ Input [1]: [c_customer_id#22] Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/extended.txt index 8aa14c43dd..0d0d80c7f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 c745a5b255..4900411443 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/explain.txt index a603d43983..6f88406582 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -13,7 +13,7 @@ TakeOrderedAndProject (45) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -35,12 +35,12 @@ TakeOrderedAndProject (45) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_address (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) @@ -57,7 +57,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] (6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] @@ -87,7 +87,7 @@ Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi @@ -100,19 +100,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] (13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#9] @@ -120,7 +120,7 @@ Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#9] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) @@ -133,19 +133,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] (20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#12] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#9] @@ -153,17 +153,17 @@ Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#9] Input [1]: [cs_ship_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#12] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(25) Filter [codegen id : 4] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -182,20 +182,20 @@ Condition : (ca_county#15 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#14, ca_county#15] Arguments: [ca_address_sk#14], [ca_address_sk#14] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [1]: [ca_address_sk#14] (31) BroadcastExchange Input [1]: [ca_address_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#14] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#14] @@ -214,24 +214,24 @@ Condition : isnotnull(cd_demo_sk#16) Input [9]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: [cd_demo_sk#16, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#17, 1, true, false, true) AS cd_gender#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#18, 1, true, false, true) AS cd_marital_status#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#19, 20, true, false, true) AS cd_education_status#27, cd_purchase_estimate#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#21, 10, true, false, true) AS cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [9]: [cd_demo_sk#16, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] (38) BroadcastExchange Input [9]: [cd_demo_sk#16, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#16] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 4] Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#16, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 4] Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [1]: [partial_count(1)] @@ -242,10 +242,10 @@ Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_pur Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#30] Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#30] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 5] Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#30] Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [1]: [count(1)] @@ -260,7 +260,7 @@ Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FI Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * CometColumnarToRow (49) ++- CometNativeColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometNativeScan parquet spark_catalog.default.date_dim (46) @@ -281,7 +281,7 @@ Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = Input [3]: [d_date_sk#9, d_year#38, d_moy#39] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt index 8935b6cc13..40a8ef4a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -22,12 +22,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,17 +49,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/simplified.txt index 0cc108b0b0..ff26eb68a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -29,17 +29,15 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,7 +48,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -61,17 +59,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/explain.txt index 28df28d9a7..5d4400f83a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#15] Input [1]: [cs_ship_customer_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#15] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#18, ca_county#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [1]: [ca_address_sk#18] (33) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#20) Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#31, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [partial_count(1)] @@ -253,10 +253,10 @@ Results [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_pur Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] Arguments: hashpartitioning(cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] @@ -271,7 +271,7 @@ Arguments: 100, [cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/simplified.txt index e7193f87e1..fd519f33e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 28df28d9a7..5d4400f83a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#15] Input [1]: [cs_ship_customer_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#15] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#18, ca_county#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [1]: [ca_address_sk#18] (33) BroadcastExchange Input [1]: [ca_address_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#4] Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#20) Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#31, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [partial_count(1)] @@ -253,10 +253,10 @@ Results [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_pur Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] Arguments: hashpartitioning(cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [1]: [count(1)] @@ -271,7 +271,7 @@ Arguments: 100, [cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index e7193f87e1..fd519f33e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/explain.txt index d697d27f18..300b750cd1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/explain.txt @@ -8,14 +8,14 @@ TakeOrderedAndProject (80) : : +- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -26,14 +26,14 @@ TakeOrderedAndProject (80) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -45,14 +45,14 @@ TakeOrderedAndProject (80) : +- BroadcastExchange (57) : +- * Filter (56) : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) + : +- CometNativeColumnarToRow (54) : +- CometColumnarExchange (53) : +- * HashAggregate (52) : +- * Project (51) : +- * BroadcastHashJoin Inner BuildRight (50) : :- * Project (48) : : +- * BroadcastHashJoin Inner BuildRight (47) - : : :- * CometColumnarToRow (42) + : : :- CometNativeColumnarToRow (42) : : : +- CometProject (41) : : : +- CometFilter (40) : : : +- CometNativeScan parquet spark_catalog.default.customer (39) @@ -63,14 +63,14 @@ TakeOrderedAndProject (80) : +- ReusedExchange (49) +- BroadcastExchange (77) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- CometColumnarExchange (74) +- * HashAggregate (73) +- * Project (72) +- * BroadcastHashJoin Inner BuildRight (71) :- * Project (69) : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * CometColumnarToRow (63) + : :- CometNativeColumnarToRow (63) : : +- CometProject (62) : : +- CometFilter (61) : : +- CometNativeScan parquet spark_catalog.default.customer (60) @@ -96,7 +96,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] (5) Scan parquet spark_catalog.default.store_sales @@ -118,30 +118,30 @@ Condition : isnotnull(ss_customer_sk#15) Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#15] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] (11) ReusedExchange [Reuses operator id: 84] Output [2]: [d_date_sk#20, d_year#21] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] @@ -152,17 +152,17 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#25, year_total#26] Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) @@ -181,7 +181,7 @@ Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] (23) Scan parquet spark_catalog.default.store_sales @@ -192,10 +192,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Condition : isnotnull(ss_customer_sk#35) @@ -203,30 +203,30 @@ Condition : isnotnull(ss_customer_sk#35) Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#27] Right keys [1]: [ss_customer_sk#35] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] (29) ReusedExchange [Reuses operator id: 88] Output [2]: [d_date_sk#40, d_year#41] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#38] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] @@ -237,10 +237,10 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] @@ -251,13 +251,13 @@ Results [3]: [c_customer_id#9 AS customer_id#44, c_preferred_cust_flag#12 AS cus Input [3]: [customer_id#44, customer_preferred_cust_flag#45, year_total#46] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#44] Join type: Inner Join condition: None -(38) Project [codegen id : 16] +(38) Project [codegen id : 12] Output [4]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46] Input [5]: [customer_id#25, year_total#26, customer_id#44, customer_preferred_cust_flag#45, year_total#46] @@ -276,7 +276,7 @@ Condition : (isnotnull(c_customer_sk#47) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] Arguments: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60], [c_customer_sk#47, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#48, 16, true, false, true) AS c_customer_id#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#49, 20, true, false, true) AS c_first_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#50, 30, true, false, true) AS c_last_name#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#51, 1, true, false, true) AS c_preferred_cust_flag#58, c_birth_country#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#53, 13, true, false, true) AS c_login#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#54, 50, true, false, true) AS c_email_address#60] -(42) CometColumnarToRow [codegen id : 10] +(42) CometNativeColumnarToRow Input [8]: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60] (43) Scan parquet spark_catalog.default.web_sales @@ -287,10 +287,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 8] +(44) ColumnarToRow [codegen id : 6] Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -(45) Filter [codegen id : 8] +(45) Filter [codegen id : 6] Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] Condition : isnotnull(ws_bill_customer_sk#61) @@ -298,30 +298,30 @@ Condition : isnotnull(ws_bill_customer_sk#61) Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#47] Right keys [1]: [ws_bill_customer_sk#61] Join type: Inner Join condition: None -(48) Project [codegen id : 10] +(48) Project [codegen id : 7] Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] Input [12]: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] (49) ReusedExchange [Reuses operator id: 84] Output [2]: [d_date_sk#65, d_year#66] -(50) BroadcastHashJoin [codegen id : 10] +(50) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#64] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(51) Project [codegen id : 10] +(51) Project [codegen id : 7] Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66] Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66] -(52) HashAggregate [codegen id : 10] +(52) HashAggregate [codegen id : 7] Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))] @@ -332,17 +332,17 @@ Results [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cus Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometColumnarToRow [codegen id : 11] +(54) CometNativeColumnarToRow Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] -(55) HashAggregate [codegen id : 11] +(55) HashAggregate [codegen id : 8] Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))#69] Results [2]: [c_customer_id#55 AS customer_id#70, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))#69,18,2) AS year_total#71] -(56) Filter [codegen id : 11] +(56) Filter [codegen id : 8] Input [2]: [customer_id#70, year_total#71] Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) @@ -350,13 +350,13 @@ Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) Input [2]: [customer_id#70, year_total#71] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(58) BroadcastHashJoin [codegen id : 16] +(58) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#70] Join type: Inner Join condition: None -(59) Project [codegen id : 16] +(59) Project [codegen id : 12] Output [5]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, year_total#71] Input [6]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, customer_id#70, year_total#71] @@ -375,7 +375,7 @@ Condition : (isnotnull(c_customer_sk#72) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] Arguments: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60], [c_customer_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#73, 16, true, false, true) AS c_customer_id#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#74, 20, true, false, true) AS c_first_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#75, 30, true, false, true) AS c_last_name#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#76, 1, true, false, true) AS c_preferred_cust_flag#58, c_birth_country#77, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#78, 13, true, false, true) AS c_login#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#79, 50, true, false, true) AS c_email_address#60] -(63) CometColumnarToRow [codegen id : 14] +(63) CometNativeColumnarToRow Input [8]: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60] (64) Scan parquet spark_catalog.default.web_sales @@ -386,10 +386,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(65) ColumnarToRow [codegen id : 12] +(65) ColumnarToRow [codegen id : 9] Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -(66) Filter [codegen id : 12] +(66) Filter [codegen id : 9] Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] Condition : isnotnull(ws_bill_customer_sk#80) @@ -397,30 +397,30 @@ Condition : isnotnull(ws_bill_customer_sk#80) Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(68) BroadcastHashJoin [codegen id : 14] +(68) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#72] Right keys [1]: [ws_bill_customer_sk#80] Join type: Inner Join condition: None -(69) Project [codegen id : 14] +(69) Project [codegen id : 10] Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] Input [12]: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] (70) ReusedExchange [Reuses operator id: 88] Output [2]: [d_date_sk#84, d_year#85] -(71) BroadcastHashJoin [codegen id : 14] +(71) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(72) Project [codegen id : 14] +(72) Project [codegen id : 10] Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#85] Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83, d_date_sk#84, d_year#85] -(73) HashAggregate [codegen id : 14] +(73) HashAggregate [codegen id : 10] Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#85] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))] @@ -431,10 +431,10 @@ Results [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cus Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(75) CometColumnarToRow [codegen id : 15] +(75) CometNativeColumnarToRow Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] -(76) HashAggregate [codegen id : 15] +(76) HashAggregate [codegen id : 11] Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))] @@ -445,13 +445,13 @@ Results [2]: [c_customer_id#55 AS customer_id#88, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#88, year_total#89] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(78) BroadcastHashJoin [codegen id : 16] +(78) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#88] Join type: Inner Join condition: (CASE WHEN (year_total#71 > 0.00) THEN (year_total#89 / year_total#71) END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#46 / year_total#26) END) -(79) Project [codegen id : 16] +(79) Project [codegen id : 12] Output [1]: [customer_preferred_cust_flag#45] Input [7]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, year_total#71, customer_id#88, year_total#89] @@ -463,7 +463,7 @@ Arguments: 100, [customer_preferred_cust_flag#45 ASC NULLS FIRST], [customer_pre Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (84) -+- * CometColumnarToRow (83) ++- CometNativeColumnarToRow (83) +- CometFilter (82) +- CometNativeScan parquet spark_catalog.default.date_dim (81) @@ -479,7 +479,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(83) CometColumnarToRow [codegen id : 1] +(83) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (84) BroadcastExchange @@ -488,7 +488,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 BroadcastExchange (88) -+- * CometColumnarToRow (87) ++- CometNativeColumnarToRow (87) +- CometFilter (86) +- CometNativeScan parquet spark_catalog.default.date_dim (85) @@ -504,7 +504,7 @@ ReadSchema: struct Input [2]: [d_date_sk#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(87) CometColumnarToRow [codegen id : 1] +(87) CometNativeColumnarToRow Input [2]: [d_date_sk#40, d_year#41] (88) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt index 0f9f19de77..47aa9517cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -24,23 +24,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -50,24 +50,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -77,19 +77,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -99,7 +99,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/simplified.txt index 6c9e276c01..80b29cda63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] @@ -8,17 +8,17 @@ TakeOrderedAndProject [customer_preferred_cust_flag] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] @@ -31,69 +31,65 @@ TakeOrderedAndProject [customer_preferred_cust_flag] Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -103,25 +99,25 @@ TakeOrderedAndProject [customer_preferred_cust_flag] ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/explain.txt index 091ad62567..937c9228e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/explain.txt @@ -420,7 +420,7 @@ Input [1]: [customer_preferred_cust_flag#42] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometFilter (75) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (77) BroadcastExchange @@ -445,7 +445,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometFilter (79) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) @@ -461,7 +461,7 @@ ReadSchema: struct Input [2]: [d_date_sk#38, d_year#39] Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#38, d_year#39] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt index 781eae9054..63858a8376 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -44,7 +44,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/simplified.txt index 4599ecf539..0cc605f69e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/simplified.txt @@ -24,11 +24,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -48,11 +46,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index 091ad62567..937c9228e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -420,7 +420,7 @@ Input [1]: [customer_preferred_cust_flag#42] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometFilter (75) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) @@ -436,7 +436,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (77) BroadcastExchange @@ -445,7 +445,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometFilter (79) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) @@ -461,7 +461,7 @@ ReadSchema: struct Input [2]: [d_date_sk#38, d_year#39] Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#38, d_year#39] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/extended.txt index 781eae9054..63858a8376 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -44,7 +44,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 4599ecf539..0cc605f69e 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 @@ -24,11 +24,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -48,11 +46,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/explain.txt index 19d16c378b..34d1d3917a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.web_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) @@ -54,37 +54,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] (21) Window Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt index 6c2a775097..15648408fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/simplified.txt index b7b0a89774..b580349644 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt index 93308784a9..0e486aa5dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt index fb83fd2f9a..c014d569f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 93308784a9..0e486aa5dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fb83fd2f9a..c014d569f4 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/explain.txt index dbacf525ef..cfff06dc67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * HashAggregate (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometColumnarExchange (34) +- * HashAggregate (33) +- * Project (32) @@ -17,22 +17,22 @@ : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) + : : : : +- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.store (4) : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometProject (12) : : : +- CometFilter (11) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) : : +- ReusedExchange (17) : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometProject (22) : +- CometFilter (21) : +- CometNativeScan parquet spark_catalog.default.customer_demographics (20) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.household_demographics (27) @@ -45,10 +45,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 1] Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 1] Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) @@ -63,20 +63,20 @@ ReadSchema: struct Input [1]: [s_store_sk#12] Condition : isnotnull(s_store_sk#12) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [s_store_sk#12] (7) BroadcastExchange Input [1]: [s_store_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 1] Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] @@ -95,33 +95,33 @@ Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) AS ca_state#16] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [2]: [ca_address_sk#13, ca_state#16] (14) BroadcastExchange Input [2]: [ca_address_sk#13, ca_state#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#3] Right keys [1]: [ca_address_sk#13] Join type: Inner Join condition: ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) -(16) Project [codegen id : 6] +(16) Project [codegen id : 1] Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] (17) ReusedExchange [Reuses operator id: 41] Output [1]: [d_date_sk#17] -(18) BroadcastHashJoin [codegen id : 6] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#10] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(19) Project [codegen id : 1] Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] @@ -140,20 +140,20 @@ Condition : (isnotnull(cd_demo_sk#18) AND ((((staticinvoke(class org.apache.spar Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] Arguments: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) AS cd_education_status#22] -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] (24) BroadcastExchange Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: ((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) -(26) Project [codegen id : 6] +(26) Project [codegen id : 1] Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22] Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] @@ -168,24 +168,24 @@ ReadSchema: struct Input [2]: [hd_demo_sk#23, hd_dep_count#24] Condition : (isnotnull(hd_demo_sk#23) AND ((hd_dep_count#24 = 3) OR (hd_dep_count#24 = 1))) -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [2]: [hd_demo_sk#23, hd_dep_count#24] (30) BroadcastExchange Input [2]: [hd_demo_sk#23, hd_dep_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#23] Join type: Inner Join condition: (((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#24 = 3)) OR (((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#24 = 1))) OR (((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#24 = 1))) -(32) Project [codegen id : 6] +(32) Project [codegen id : 1] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22, hd_demo_sk#23, hd_dep_count#24] -(33) HashAggregate [codegen id : 6] +(33) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] Keys: [] Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] @@ -196,10 +196,10 @@ Results [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] -(36) HashAggregate [codegen id : 7] +(36) HashAggregate [codegen id : 2] Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] @@ -210,7 +210,7 @@ Results [4]: [avg(ss_quantity#5)#39 AS avg(ss_quantity)#43, cast((avg(UnscaledVa Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (41) -+- * CometColumnarToRow (40) ++- CometNativeColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometNativeScan parquet spark_catalog.default.date_dim (37) @@ -231,7 +231,7 @@ Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#17, d_year#47] Arguments: [d_date_sk#17], [d_date_sk#17] -(40) CometColumnarToRow [codegen id : 1] +(40) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (41) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt index 4c0d0b7a33..9ff51aab85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,31 +17,31 @@ HashAggregate : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/simplified.txt index a33ae5a161..d7dc065e83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) +WholeStageCodegen (2) HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (6) + WholeStageCodegen (1) HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] @@ -21,41 +21,31 @@ WholeStageCodegen (7) Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometNativeColumnarToRow + CometFilter [hd_demo_sk,hd_dep_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/explain.txt index 74da8ba883..d4881d2dff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/explain.txt @@ -200,7 +200,7 @@ Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesal Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -221,7 +221,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#17, d_year#18] Arguments: [d_date_sk#17], [d_date_sk#17] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt index 08e9beb692..5568227543 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/simplified.txt index 8ef882a435..3d2ac96c74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index 74da8ba883..d4881d2dff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -200,7 +200,7 @@ Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesal Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (39) -+- * CometColumnarToRow (38) ++- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) @@ -221,7 +221,7 @@ Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#17, d_year#18] Arguments: [d_date_sk#17], [d_date_sk#17] -(38) CometColumnarToRow [codegen id : 1] +(38) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (39) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/extended.txt index 08e9beb692..5568227543 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 8ef882a435..3d2ac96c74 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/explain.txt index 8abc7aabfb..801ba7d85d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (110) +- * HashAggregate (109) - +- * CometColumnarToRow (108) + +- CometNativeColumnarToRow (108) +- CometColumnarExchange (107) +- * HashAggregate (106) +- * Expand (105) @@ -9,7 +9,7 @@ TakeOrderedAndProject (110) :- * Project (69) : +- * Filter (68) : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) + : +- CometNativeColumnarToRow (66) : +- CometColumnarExchange (65) : +- * HashAggregate (64) : +- * Project (63) @@ -23,12 +23,12 @@ TakeOrderedAndProject (110) : : : +- BroadcastExchange (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) + : : : :- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (48) : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) + : : : :- CometNativeColumnarToRow (36) : : : : +- CometHashAggregate (35) : : : : +- CometColumnarExchange (34) : : : : +- * HashAggregate (33) @@ -41,7 +41,7 @@ TakeOrderedAndProject (110) : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : +- BroadcastExchange (27) : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) + : : : : : :- CometNativeColumnarToRow (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- BroadcastExchange (25) @@ -53,7 +53,7 @@ TakeOrderedAndProject (110) : : : : : : : +- * ColumnarToRow (14) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometNativeColumnarToRow (18) : : : : : : +- CometFilter (17) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : +- ReusedExchange (22) @@ -70,7 +70,7 @@ TakeOrderedAndProject (110) : : : +- ReusedExchange (43) : : +- BroadcastExchange (58) : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) + : : :- CometNativeColumnarToRow (55) : : : +- CometFilter (54) : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : +- ReusedExchange (56) @@ -78,7 +78,7 @@ TakeOrderedAndProject (110) :- * Project (86) : +- * Filter (85) : +- * HashAggregate (84) - : +- * CometColumnarToRow (83) + : +- CometNativeColumnarToRow (83) : +- CometColumnarExchange (82) : +- * HashAggregate (81) : +- * Project (80) @@ -95,7 +95,7 @@ TakeOrderedAndProject (110) +- * Project (103) +- * Filter (102) +- * HashAggregate (101) - +- * CometColumnarToRow (100) + +- CometNativeColumnarToRow (100) +- CometColumnarExchange (99) +- * HashAggregate (98) +- * Project (97) @@ -119,10 +119,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -137,7 +137,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -166,7 +166,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -195,33 +195,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -326,7 +326,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -336,13 +336,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -350,7 +350,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -367,13 +367,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(69) Project [codegen id : 26] +(69) Project [codegen id : 15] Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#37 AS i_brand_id#54, i_class_id#38 AS i_class_id#55, i_category_id#39 AS i_category_id#56] Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] @@ -443,17 +443,17 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 51] +(71) ColumnarToRow [codegen id : 29] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -(72) Filter [codegen id : 51] +(72) Filter [codegen id : 29] Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] Condition : isnotnull(cs_item_sk#57) (73) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#35] -(74) BroadcastHashJoin [codegen id : 51] +(74) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#57] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -462,30 +462,30 @@ Join condition: None (75) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -(76) BroadcastHashJoin [codegen id : 51] +(76) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#57] Right keys [1]: [i_item_sk#61] Join type: Inner Join condition: None -(77) Project [codegen id : 51] +(77) Project [codegen id : 29] Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#62, i_class_id#63, i_category_id#64] Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] (78) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#65] -(79) BroadcastHashJoin [codegen id : 51] +(79) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_sold_date_sk#60] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(80) Project [codegen id : 51] +(80) Project [codegen id : 29] Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#62, i_class_id#63, i_category_id#64] Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -(81) HashAggregate [codegen id : 51] +(81) HashAggregate [codegen id : 29] Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#62, i_class_id#63, i_category_id#64] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] @@ -496,21 +496,21 @@ Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70 Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(83) CometColumnarToRow [codegen id : 52] +(83) CometNativeColumnarToRow Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -(84) HashAggregate [codegen id : 52] +(84) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72, count(1)#73] Results [5]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72 AS sales#74, count(1)#73 AS number_sales#75] -(85) Filter [codegen id : 52] +(85) Filter [codegen id : 30] Input [5]: [i_brand_id#62, i_class_id#63, i_category_id#64, sales#74, number_sales#75] Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(86) Project [codegen id : 52] +(86) Project [codegen id : 30] Output [6]: [sales#74, number_sales#75, catalog AS channel#76, i_brand_id#62, i_class_id#63, i_category_id#64] Input [5]: [i_brand_id#62, i_class_id#63, i_category_id#64, sales#74, number_sales#75] @@ -522,17 +522,17 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(88) ColumnarToRow [codegen id : 77] +(88) ColumnarToRow [codegen id : 44] Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -(89) Filter [codegen id : 77] +(89) Filter [codegen id : 44] Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] Condition : isnotnull(ws_item_sk#77) (90) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#35] -(91) BroadcastHashJoin [codegen id : 77] +(91) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#77] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -541,30 +541,30 @@ Join condition: None (92) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] -(93) BroadcastHashJoin [codegen id : 77] +(93) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#77] Right keys [1]: [i_item_sk#81] Join type: Inner Join condition: None -(94) Project [codegen id : 77] +(94) Project [codegen id : 44] Output [6]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#82, i_class_id#83, i_category_id#84] Input [8]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] (95) ReusedExchange [Reuses operator id: 135] Output [1]: [d_date_sk#85] -(96) BroadcastHashJoin [codegen id : 77] +(96) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#80] Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(97) Project [codegen id : 77] +(97) Project [codegen id : 44] Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#82, i_class_id#83, i_category_id#84] Input [7]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#82, i_class_id#83, i_category_id#84, d_date_sk#85] -(98) HashAggregate [codegen id : 77] +(98) HashAggregate [codegen id : 44] Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#82, i_class_id#83, i_category_id#84] Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)] @@ -575,31 +575,31 @@ Results [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90 Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] Arguments: hashpartitioning(i_brand_id#82, i_class_id#83, i_category_id#84, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(100) CometColumnarToRow [codegen id : 78] +(100) CometNativeColumnarToRow Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] -(101) HashAggregate [codegen id : 78] +(101) HashAggregate [codegen id : 45] Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92, count(1)#93] Results [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92 AS sales#94, count(1)#93 AS number_sales#95] -(102) Filter [codegen id : 78] +(102) Filter [codegen id : 45] Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#94, number_sales#95] Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(103) Project [codegen id : 78] +(103) Project [codegen id : 45] Output [6]: [sales#94, number_sales#95, web AS channel#96, i_brand_id#82, i_class_id#83, i_category_id#84] Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#94, number_sales#95] (104) Union -(105) Expand [codegen id : 79] +(105) Expand [codegen id : 46] Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] -(106) HashAggregate [codegen id : 79] +(106) HashAggregate [codegen id : 46] Input [7]: [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] @@ -610,10 +610,10 @@ Results [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(108) CometColumnarToRow [codegen id : 80] +(108) CometNativeColumnarToRow Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] -(109) HashAggregate [codegen id : 80] +(109) HashAggregate [codegen id : 47] Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] Functions [2]: [sum(sales#49), sum(number_sales#50)] @@ -628,7 +628,7 @@ Arguments: 100, [channel#97 ASC NULLS FIRST, i_brand_id#98 ASC NULLS FIRST, i_cl Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * HashAggregate (130) -+- * CometColumnarToRow (129) ++- CometNativeColumnarToRow (129) +- CometColumnarExchange (128) +- * HashAggregate (127) +- Union (126) @@ -656,19 +656,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#114), dynamicpruningexpression(ss_sold_date_sk#114 IN dynamicpruning#12)] ReadSchema: struct -(112) ColumnarToRow [codegen id : 2] +(112) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114] (113) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#115] -(114) BroadcastHashJoin [codegen id : 2] +(114) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#114] Right keys [1]: [d_date_sk#115] Join type: Inner Join condition: None -(115) Project [codegen id : 2] +(115) Project [codegen id : 1] Output [2]: [ss_quantity#112 AS quantity#116, ss_list_price#113 AS list_price#117] Input [4]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114, d_date_sk#115] @@ -679,19 +679,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#120), dynamicpruningexpression(cs_sold_date_sk#120 IN dynamicpruning#12)] ReadSchema: struct -(117) ColumnarToRow [codegen id : 4] +(117) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120] (118) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#121] -(119) BroadcastHashJoin [codegen id : 4] +(119) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#120] Right keys [1]: [d_date_sk#121] Join type: Inner Join condition: None -(120) Project [codegen id : 4] +(120) Project [codegen id : 2] Output [2]: [cs_quantity#118 AS quantity#122, cs_list_price#119 AS list_price#123] Input [4]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120, d_date_sk#121] @@ -702,25 +702,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#126), dynamicpruningexpression(ws_sold_date_sk#126 IN dynamicpruning#12)] ReadSchema: struct -(122) ColumnarToRow [codegen id : 6] +(122) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126] (123) ReusedExchange [Reuses operator id: 140] Output [1]: [d_date_sk#127] -(124) BroadcastHashJoin [codegen id : 6] +(124) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#126] Right keys [1]: [d_date_sk#127] Join type: Inner Join condition: None -(125) Project [codegen id : 6] +(125) Project [codegen id : 3] Output [2]: [ws_quantity#124 AS quantity#128, ws_list_price#125 AS list_price#129] Input [4]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126, d_date_sk#127] (126) Union -(127) HashAggregate [codegen id : 7] +(127) HashAggregate [codegen id : 4] Input [2]: [quantity#116, list_price#117] Keys: [] Functions [1]: [partial_avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] @@ -731,10 +731,10 @@ Results [2]: [sum#132, count#133] Input [2]: [sum#132, count#133] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(129) CometColumnarToRow [codegen id : 8] +(129) CometNativeColumnarToRow Input [2]: [sum#132, count#133] -(130) HashAggregate [codegen id : 8] +(130) HashAggregate [codegen id : 5] Input [2]: [sum#132, count#133] Keys: [] Functions [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] @@ -749,7 +749,7 @@ Subquery:4 Hosting operator id = 121 Hosting Expression = ws_sold_date_sk#126 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (135) -+- * CometColumnarToRow (134) ++- CometNativeColumnarToRow (134) +- CometProject (133) +- CometFilter (132) +- CometNativeScan parquet spark_catalog.default.date_dim (131) @@ -770,7 +770,7 @@ Condition : ((((isnotnull(d_year#136) AND isnotnull(d_moy#137)) AND (d_year#136 Input [3]: [d_date_sk#40, d_year#136, d_moy#137] Arguments: [d_date_sk#40], [d_date_sk#40] -(134) CometColumnarToRow [codegen id : 1] +(134) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (135) BroadcastExchange @@ -779,7 +779,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (140) -+- * CometColumnarToRow (139) ++- CometNativeColumnarToRow (139) +- CometProject (138) +- CometFilter (137) +- CometNativeScan parquet spark_catalog.default.date_dim (136) @@ -800,7 +800,7 @@ Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 < Input [2]: [d_date_sk#24, d_year#138] Arguments: [d_date_sk#24], [d_date_sk#24] -(139) CometColumnarToRow [codegen id : 1] +(139) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (140) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt index dfd3434d90..223593e3d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -9,7 +9,7 @@ TakeOrderedAndProject : +- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,12 +39,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -57,19 +57,19 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -82,13 +82,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -101,16 +101,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -124,28 +124,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -158,13 +158,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -177,16 +177,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -200,16 +200,16 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -217,7 +217,7 @@ TakeOrderedAndProject : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -232,12 +232,12 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -250,13 +250,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -269,16 +269,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -292,28 +292,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -326,13 +326,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -345,16 +345,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -368,16 +368,16 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -385,7 +385,7 @@ TakeOrderedAndProject +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -400,12 +400,12 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -418,13 +418,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -437,16 +437,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -460,28 +460,28 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -494,13 +494,13 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -513,16 +513,16 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -536,16 +536,16 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/simplified.txt index c989fe9a81..f348396ac0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/simplified.txt @@ -1,28 +1,28 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) + WholeStageCodegen (47) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) + WholeStageCodegen (46) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (26) + WholeStageCodegen (15) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #3 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #13 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -31,7 +31,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -40,7 +40,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,10 +50,10 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter ReusedExchange [d_date_sk] #7 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -66,30 +66,28 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (11) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (10) + WholeStageCodegen (5) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [brand_id,class_id,category_id] CometColumnarExchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -101,23 +99,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) + WholeStageCodegen (2) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (1) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -129,18 +125,16 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedSubquery [d_date_sk] #2 InputAdapter BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 - WholeStageCodegen (9) + WholeStageCodegen (4) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -156,25 +150,25 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #12 - WholeStageCodegen (23) + WholeStageCodegen (13) BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [ss_item_sk] #4 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) + WholeStageCodegen (30) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) + WholeStageCodegen (29) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -192,15 +186,15 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) + WholeStageCodegen (45) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sales] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/explain.txt index 0289abc42d..b41871f037 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/explain.txt @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#115 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * CometColumnarToRow (128) ++- CometNativeColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#123) AND (d_year#123 >= 1999)) AND (d_year#123 < Input [2]: [d_date_sk#26, d_year#123] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) CometColumnarToRow [codegen id : 1] +(128) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (129) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt index 4af04a7846..49ad7403ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -213,7 +213,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -277,7 +277,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -355,7 +355,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -419,7 +419,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/simplified.txt index 79c782f2ca..fea1950918 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/simplified.txt @@ -44,12 +44,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #4 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -68,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 0289abc42d..b41871f037 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#115 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * CometColumnarToRow (128) ++- CometNativeColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#123) AND (d_year#123 >= 1999)) AND (d_year#123 < Input [2]: [d_date_sk#26, d_year#123] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) CometColumnarToRow [codegen id : 1] +(128) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (129) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/extended.txt index 4af04a7846..49ad7403ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -213,7 +213,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -277,7 +277,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -355,7 +355,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -419,7 +419,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 79c782f2ca..fea1950918 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 @@ -44,12 +44,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #4 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -68,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #8 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/explain.txt index d539836be8..21039119a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (87) +- * BroadcastHashJoin Inner BuildRight (86) :- * Filter (68) : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) + : +- CometNativeColumnarToRow (66) : +- CometColumnarExchange (65) : +- * HashAggregate (64) : +- * Project (63) @@ -17,12 +17,12 @@ TakeOrderedAndProject (87) : : : +- BroadcastExchange (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) + : : : :- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (48) : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) + : : : :- CometNativeColumnarToRow (36) : : : : +- CometHashAggregate (35) : : : : +- CometColumnarExchange (34) : : : : +- * HashAggregate (33) @@ -35,7 +35,7 @@ TakeOrderedAndProject (87) : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : +- BroadcastExchange (27) : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) + : : : : : :- CometNativeColumnarToRow (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- BroadcastExchange (25) @@ -47,7 +47,7 @@ TakeOrderedAndProject (87) : : : : : : : +- * ColumnarToRow (14) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometNativeColumnarToRow (18) : : : : : : +- CometFilter (17) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : +- ReusedExchange (22) @@ -64,7 +64,7 @@ TakeOrderedAndProject (87) : : : +- ReusedExchange (43) : : +- BroadcastExchange (58) : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) + : : :- CometNativeColumnarToRow (55) : : : +- CometFilter (54) : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : +- ReusedExchange (56) @@ -72,7 +72,7 @@ TakeOrderedAndProject (87) +- BroadcastExchange (85) +- * Filter (84) +- * HashAggregate (83) - +- * CometColumnarToRow (82) + +- CometNativeColumnarToRow (82) +- CometColumnarExchange (81) +- * HashAggregate (80) +- * Project (79) @@ -96,10 +96,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -114,7 +114,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -143,7 +143,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -172,33 +172,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -303,7 +303,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -313,13 +313,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -327,7 +327,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -344,13 +344,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -416,17 +416,17 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 50] +(70) ColumnarToRow [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(71) Filter [codegen id : 50] +(71) Filter [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) (72) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#35] -(73) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -435,30 +435,30 @@ Join condition: None (74) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(75) BroadcastHashJoin [codegen id : 50] +(75) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(76) Project [codegen id : 50] +(76) Project [codegen id : 28] Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] (77) ReusedExchange [Reuses operator id: 126] Output [1]: [d_date_sk#63] -(78) BroadcastHashJoin [codegen id : 50] +(78) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(79) Project [codegen id : 50] +(79) Project [codegen id : 28] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] -(80) HashAggregate [codegen id : 50] +(80) HashAggregate [codegen id : 28] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] @@ -469,17 +469,17 @@ Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(82) CometColumnarToRow [codegen id : 51] +(82) CometNativeColumnarToRow Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -(83) HashAggregate [codegen id : 51] +(83) HashAggregate [codegen id : 29] Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(84) Filter [codegen id : 51] +(84) Filter [codegen id : 29] Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -487,7 +487,7 @@ Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(Reu Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 52] +(86) BroadcastHashJoin [codegen id : 30] Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join type: Inner @@ -501,7 +501,7 @@ Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * HashAggregate (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometColumnarExchange (105) +- * HashAggregate (104) +- Union (103) @@ -529,19 +529,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 2] +(89) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] (90) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#78] -(91) BroadcastHashJoin [codegen id : 2] +(91) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(92) Project [codegen id : 2] +(92) Project [codegen id : 1] Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] @@ -552,19 +552,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 4] +(94) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] (95) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#84] -(96) BroadcastHashJoin [codegen id : 4] +(96) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(97) Project [codegen id : 4] +(97) Project [codegen id : 2] Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] @@ -575,25 +575,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] ReadSchema: struct -(99) ColumnarToRow [codegen id : 6] +(99) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] (100) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#90] -(101) BroadcastHashJoin [codegen id : 6] +(101) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#89] Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(102) Project [codegen id : 6] +(102) Project [codegen id : 3] Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] (103) Union -(104) HashAggregate [codegen id : 7] +(104) HashAggregate [codegen id : 4] Input [2]: [quantity#79, list_price#80] Keys: [] Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] @@ -604,10 +604,10 @@ Results [2]: [sum#95, count#96] Input [2]: [sum#95, count#96] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(106) CometColumnarToRow [codegen id : 8] +(106) CometNativeColumnarToRow Input [2]: [sum#95, count#96] -(107) HashAggregate [codegen id : 8] +(107) HashAggregate [codegen id : 5] Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] @@ -622,7 +622,7 @@ Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#89 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (112) -+- * CometColumnarToRow (111) ++- CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometNativeScan parquet spark_catalog.default.date_dim (108) @@ -643,7 +643,7 @@ Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subq Input [2]: [d_date_sk#40, d_week_seq#99] Arguments: [d_date_sk#40], [d_date_sk#40] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (112) BroadcastExchange @@ -651,7 +651,7 @@ Input [1]: [d_date_sk#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] Subquery:6 Hosting operator id = 109 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* CometColumnarToRow (116) +CometNativeColumnarToRow (116) +- CometProject (115) +- CometFilter (114) +- CometNativeScan parquet spark_catalog.default.date_dim (113) @@ -672,12 +672,12 @@ Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Arguments: [d_week_seq#102], [d_week_seq#102] -(116) CometColumnarToRow [codegen id : 1] +(116) CometNativeColumnarToRow Input [1]: [d_week_seq#102] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometNativeScan parquet spark_catalog.default.date_dim (117) @@ -698,7 +698,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 < Input [2]: [d_date_sk#24, d_year#103] Arguments: [d_date_sk#24], [d_date_sk#24] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (121) BroadcastExchange @@ -713,7 +713,7 @@ Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquer Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 BroadcastExchange (126) -+- * CometColumnarToRow (125) ++- CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometNativeScan parquet spark_catalog.default.date_dim (122) @@ -734,7 +734,7 @@ Condition : ((isnotnull(d_week_seq#106) AND (d_week_seq#106 = Subquery scalar-su Input [2]: [d_date_sk#63, d_week_seq#106] Arguments: [d_date_sk#63], [d_date_sk#63] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_date_sk#63] (126) BroadcastExchange @@ -742,7 +742,7 @@ Input [1]: [d_date_sk#63] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:12 Hosting operator id = 123 Hosting Expression = Subquery scalar-subquery#107, [id=#108] -* CometColumnarToRow (130) +CometNativeColumnarToRow (130) +- CometProject (129) +- CometFilter (128) +- CometNativeScan parquet spark_catalog.default.date_dim (127) @@ -763,7 +763,7 @@ Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Arguments: [d_week_seq#102], [d_week_seq#102] -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [1]: [d_week_seq#102] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt index c4138c254a..7921f2fa61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject :- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -23,7 +23,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -33,12 +33,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -51,11 +51,11 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,12 +63,12 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -81,13 +81,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -100,16 +100,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,28 +123,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -157,13 +157,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -176,16 +176,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -199,20 +199,20 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -221,7 +221,7 @@ TakeOrderedAndProject +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -234,11 +234,11 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -246,12 +246,12 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -264,13 +264,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -283,16 +283,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -306,28 +306,28 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -340,13 +340,13 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -359,16 +359,16 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -382,20 +382,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/simplified.txt index b0eae963c3..45d526a696 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (30) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #12 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -21,7 +21,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #6 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -56,37 +56,33 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (5) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [brand_id,class_id,category_id] CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -98,23 +94,21 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 - WholeStageCodegen (4) + WholeStageCodegen (2) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #8 - WholeStageCodegen (3) + WholeStageCodegen (1) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -126,18 +120,16 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 - WholeStageCodegen (9) + WholeStageCodegen (4) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -153,10 +145,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 - WholeStageCodegen (23) + WholeStageCodegen (13) BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter @@ -165,14 +157,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (51) + WholeStageCodegen (29) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + WholeStageCodegen (28) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -185,19 +177,15 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/explain.txt index bcef4db073..1ce5573eac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#98] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#99) AND (d_year#99 >= 1999)) AND (d_year#99 <= 2 Input [2]: [d_date_sk#26, d_year#99] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subq Input [2]: [d_date_sk#64, d_week_seq#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#64] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#98] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt index f56d229b68..c1b791f357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt @@ -44,11 +44,11 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,11 +197,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -224,7 +224,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,7 +288,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/simplified.txt index fb9abae378..71f9d72e7d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/simplified.txt @@ -39,19 +39,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -131,19 +125,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index bcef4db073..1ce5573eac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#98] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#99) AND (d_year#99 >= 1999)) AND (d_year#99 <= 2 Input [2]: [d_date_sk#26, d_year#99] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subq Input [2]: [d_date_sk#64, d_week_seq#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#64] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#98] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/extended.txt index f56d229b68..c1b791f357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/extended.txt @@ -44,11 +44,11 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,11 +197,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -224,7 +224,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,7 +288,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fb9abae378..71f9d72e7d 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 @@ -39,19 +39,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -131,19 +125,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/explain.txt index 594939616e..a9b46e19b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -14,11 +14,11 @@ TakeOrderedAndProject (24) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) + : : +- CometNativeColumnarToRow (6) : : +- CometFilter (5) : : +- CometNativeScan parquet spark_catalog.default.customer (4) : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) + : +- CometNativeColumnarToRow (13) : +- CometProject (12) : +- CometFilter (11) : +- CometNativeScan parquet spark_catalog.default.customer_address (10) @@ -33,10 +33,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_bill_customer_sk#1) @@ -51,20 +51,20 @@ ReadSchema: struct Input [2]: [c_customer_sk#5, c_current_addr_sk#6] Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [2]: [c_customer_sk#5, c_current_addr_sk#6] (7) BroadcastExchange Input [2]: [c_customer_sk#5, c_current_addr_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#5] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] @@ -83,37 +83,37 @@ Condition : isnotnull(ca_address_sk#7) Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#9, 10, true, false, true) AS ca_zip#11] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] (14) BroadcastExchange Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#6] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) -(16) Project [codegen id : 4] +(16) Project [codegen id : 1] Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] (17) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#12] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [2]: [cs_sales_price#2, ca_zip#11] Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [2]: [cs_sales_price#2, ca_zip#11] Keys [1]: [ca_zip#11] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] @@ -124,10 +124,10 @@ Results [2]: [ca_zip#11, sum#14] Input [2]: [ca_zip#11, sum#14] Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [2]: [ca_zip#11, sum#14] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [2]: [ca_zip#11, sum#14] Keys [1]: [ca_zip#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] @@ -142,7 +142,7 @@ Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : ((((isnotnull(d_qoy#18) AND isnotnull(d_year#17)) AND (d_qoy#18 = 2) Input [3]: [d_date_sk#12, d_year#17, d_qoy#18] Arguments: [d_date_sk#12], [d_date_sk#12] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#12] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt index 8ed98bc5cb..d0d170db53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -14,21 +14,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/simplified.txt index f31442dcfe..0c2174ad74 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_zip] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [ca_zip,cs_sales_price] [sum,sum] Project [cs_sales_price,ca_zip] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -18,26 +18,20 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] + CometFilter [ca_address_sk,ca_state,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/explain.txt index af9b2efbd1..d7590ade8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/explain.txt @@ -142,7 +142,7 @@ Input [2]: [ca_zip#11, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2) Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#12] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt index 6de0c64850..55382845fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/simplified.txt index c39b96efe3..97974aec5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/simplified.txt @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt index af9b2efbd1..d7590ade8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt @@ -142,7 +142,7 @@ Input [2]: [ca_zip#11, sum(cs_sales_price)#16] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2) Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] Arguments: [d_date_sk#12], [d_date_sk#12] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#12] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/extended.txt index 6de0c64850..55382845fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 c39b96efe3..97974aec5d 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 @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/explain.txt index 2a3c8932ec..49499b4b09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (41) +CometNativeColumnarToRow (41) +- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -204,7 +204,7 @@ Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [3]: [cs_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -230,6 +230,6 @@ Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -(41) CometColumnarToRow [codegen id : 2] +(41) CometNativeColumnarToRow Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/extended.txt index 9889eeaca3..cb53684dcf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/extended.txt @@ -1,9 +1,9 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/simplified.txt index 429d83d08c..3254fa6c33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/simplified.txt @@ -1,45 +1,43 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_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] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_warehouse_sk] - CometSort [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometExchange [cs_order_number] #2 - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] +CometNativeColumnarToRow + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + 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] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] + CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] + CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_order_number,cs_warehouse_sk] + CometSort [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometExchange [cs_order_number] #2 + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometFilter [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + CometSort [cs_warehouse_sk,cs_order_number] + CometExchange [cs_order_number] #3 + CometProject [cs_warehouse_sk,cs_order_number] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + CometSort [cr_order_number] + CometExchange [cr_order_number] #4 + CometProject [cr_order_number] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cc_call_center_sk] #7 + CometProject [cc_call_center_sk] + CometFilter [cc_call_center_sk,cc_county] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/explain.txt index 5005b676fc..f4f0b9e155 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- ReusedExchange (19) : : +- ReusedExchange (22) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.store (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.item (32) @@ -52,10 +52,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) @@ -78,13 +78,13 @@ Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnot Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 8] +(8) BroadcastHashJoin [codegen id : 3] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] Join type: Inner Join condition: None -(9) Project [codegen id : 8] +(9) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] @@ -107,52 +107,52 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 3] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] Join type: Inner Join condition: None -(15) Project [codegen id : 8] +(15) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] (16) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#18] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(18) Project [codegen id : 3] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#18] (19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#19] -(20) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_returned_date_sk#12] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(21) Project [codegen id : 8] +(21) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] (22) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(24) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] @@ -171,20 +171,20 @@ Condition : isnotnull(s_store_sk#21) Input [2]: [s_store_sk#21, s_state#22] Arguments: [s_store_sk#21, s_state#23], [s_store_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#22, 2, true, false, true) AS s_state#23] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [2]: [s_store_sk#21, s_state#23] (29) BroadcastExchange Input [2]: [s_store_sk#21, s_state#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#21, s_state#23] @@ -203,24 +203,24 @@ Condition : isnotnull(i_item_sk#24) Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] Arguments: [i_item_sk#24, i_item_id#27, i_item_desc#26], [i_item_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#25, 16, true, false, true) AS i_item_id#27, i_item_desc#26] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] (36) BroadcastExchange Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#24] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(38) Project [codegen id : 3] Output [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_sk#24, i_item_id#27, i_item_desc#26] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 3] Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] @@ -231,10 +231,10 @@ Results [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] Arguments: hashpartitioning(i_item_id#27, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 4] Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] @@ -249,7 +249,7 @@ Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [d_date_sk#18, d_quarter_name#85] Arguments: [d_date_sk#18], [d_date_sk#18] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#18] (48) BroadcastExchange @@ -279,7 +279,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -300,7 +300,7 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [d_date_sk#19, d_quarter_name#86] Arguments: [d_date_sk#19], [d_date_sk#19] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt index 26e924fd7e..15c6bec5dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,27 +42,27 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/simplified.txt index c63dd716a1..591d904435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,12 +39,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -63,17 +59,13 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk,i_item_id,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/explain.txt index f502a5cd8d..0ec04f50e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/explain.txt @@ -256,7 +256,7 @@ Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#21] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/simplified.txt index 1d48f96b09..a274ade9c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/simplified.txt @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index f502a5cd8d..0ec04f50e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -256,7 +256,7 @@ Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#21] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 1d48f96b09..a274ade9c5 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 @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_quarter_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/explain.txt index 0503116eb3..8c843bd4f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (46) +- * HashAggregate (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometColumnarExchange (43) +- * HashAggregate (42) +- * Expand (41) @@ -21,27 +21,27 @@ TakeOrderedAndProject (46) : : : : : : +- * ColumnarToRow (2) : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : +- BroadcastExchange (8) - : : : : : +- * CometColumnarToRow (7) + : : : : : +- CometNativeColumnarToRow (7) : : : : : +- CometProject (6) : : : : : +- CometFilter (5) : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : : : +- BroadcastExchange (15) - : : : : +- * CometColumnarToRow (14) + : : : : +- CometNativeColumnarToRow (14) : : : : +- CometProject (13) : : : : +- CometFilter (12) : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) : : : +- BroadcastExchange (21) - : : : +- * CometColumnarToRow (20) + : : : +- CometNativeColumnarToRow (20) : : : +- CometFilter (19) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) : : +- BroadcastExchange (28) - : : +- * CometColumnarToRow (27) + : : +- CometNativeColumnarToRow (27) : : +- CometProject (26) : : +- CometFilter (25) : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) : +- ReusedExchange (31) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.item (34) @@ -55,10 +55,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) @@ -77,20 +77,20 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [cd_demo_sk#11, cd_dep_count#14] (8) BroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 1] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] @@ -109,20 +109,20 @@ Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15 Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) CometColumnarToRow [codegen id : 2] +(14) CometNativeColumnarToRow Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] (15) BroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 7] +(16) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(17) Project [codegen id : 7] +(17) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] @@ -137,20 +137,20 @@ ReadSchema: struct Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [1]: [cd_demo_sk#20] (21) BroadcastExchange Input [1]: [cd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 1] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -169,33 +169,33 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] -(27) CometColumnarToRow [codegen id : 4] +(27) CometNativeColumnarToRow Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (28) BroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(30) Project [codegen id : 7] +(30) Project [codegen id : 1] Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (31) ReusedExchange [Reuses operator id: 51] Output [1]: [d_date_sk#26] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] @@ -214,28 +214,28 @@ Condition : isnotnull(i_item_sk#27) Input [2]: [i_item_sk#27, i_item_id#28] Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] -(37) CometColumnarToRow [codegen id : 6] +(37) CometNativeColumnarToRow Input [2]: [i_item_sk#27, i_item_id#29] (38) BroadcastExchange Input [2]: [i_item_sk#27, i_item_id#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 7] +(39) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(40) Project [codegen id : 7] +(40) Project [codegen id : 1] Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] -(41) Expand [codegen id : 7] +(41) Expand [codegen id : 1] Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -(42) HashAggregate [codegen id : 7] +(42) HashAggregate [codegen id : 1] Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] @@ -246,10 +246,10 @@ Results [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_gro Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(44) CometColumnarToRow [codegen id : 8] +(44) CometNativeColumnarToRow Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -(45) HashAggregate [codegen id : 8] +(45) HashAggregate [codegen id : 2] Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] @@ -264,7 +264,7 @@ Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (51) -+- * CometColumnarToRow (50) ++- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometNativeScan parquet spark_catalog.default.date_dim (47) @@ -285,7 +285,7 @@ Condition : ((isnotnull(d_year#77) AND (d_year#77 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#77] Arguments: [d_date_sk#26], [d_date_sk#26] -(50) CometColumnarToRow [codegen id : 1] +(50) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt index b47fce49b3..53fa7b6342 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -21,36 +21,36 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/simplified.txt index 77a45c46cd..0162997d1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] @@ -25,50 +25,38 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeColumnarToRow + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeColumnarToRow + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometNativeColumnarToRow + CometFilter [cd_demo_sk] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/explain.txt index 5fefd21bfa..87a8904c51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/explain.txt @@ -249,7 +249,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, ag Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt index 0b554c7e7c..7958390823 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/simplified.txt index 6c2b8b2e4f..6abfbb11b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index 5fefd21bfa..87a8904c51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -249,7 +249,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, ag Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/extended.txt index 0b554c7e7c..7958390823 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6c2b8b2e4f..6abfbb11b2 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 @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/explain.txt index 539af26bc5..6465f46bc9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (36) +CometNativeColumnarToRow (36) +- CometTakeOrderedAndProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -203,6 +203,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt index fbe6798e3d..fa37da7c94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/simplified.txt index 675500cd0b..f6eefbf9a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/simplified.txt @@ -1,38 +1,36 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_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 [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] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] - CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] - CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_zip] +CometNativeColumnarToRow + CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_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 [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] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk,ca_address_sk,ca_zip] + CometProject [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + CometBroadcastHashJoin [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_customer_sk,c_current_addr_sk] + CometProject [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 + CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometBroadcastExchange [ca_address_sk,ca_zip] #5 + CometProject [ca_zip] [ca_address_sk,ca_zip] + CometFilter [ca_address_sk,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [s_store_sk,s_zip] #6 + CometProject [s_zip] [s_store_sk,s_zip] + CometFilter [s_store_sk,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/explain.txt index 22fbab2817..9f109f4812 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (34) +CometNativeColumnarToRow (34) +- CometSort (33) +- CometExchange (32) +- CometProject (31) @@ -188,6 +188,6 @@ Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREM 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: [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 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow 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.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/extended.txt index 7e6f4d3a3c..c3d8dd11af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/simplified.txt index e4b6e81639..6d51b41514 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/simplified.txt @@ -1,36 +1,34 @@ -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)] - 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] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - 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_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] - CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [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] - CometFilter [d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] +CometNativeColumnarToRow + 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)] + 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] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + 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_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] + CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [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] + CometFilter [d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/explain.txt index c699bf18de..e0c7fc73ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.catalog_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) @@ -54,37 +54,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] (21) Window Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt index 369ec68bb4..6634edcbed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/simplified.txt index fb0ed62abe..21e87e03b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt index c23383e09a..231e3847e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt index 2958d060fe..b3a099fb40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index c23383e09a..231e3847e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 2958d060fe..b3a099fb40 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/explain.txt index 65cbe8c435..69c85a6649 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (25) +- * Filter (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -15,11 +15,11 @@ TakeOrderedAndProject (25) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.inventory (1) : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) + : : +- CometNativeColumnarToRow (6) : : +- CometFilter (5) : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) + : +- CometNativeColumnarToRow (13) : +- CometProject (12) : +- CometFilter (11) : +- CometNativeScan parquet spark_catalog.default.item (10) @@ -34,10 +34,10 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) @@ -52,20 +52,20 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Condition : isnotnull(w_warehouse_sk#6) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] (7) BroadcastExchange Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#6] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] @@ -84,37 +84,37 @@ Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) A Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#9, 16, true, false, true) AS i_item_id#11] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [2]: [i_item_sk#8, i_item_id#11] (14) BroadcastExchange Input [2]: [i_item_sk#8, i_item_id#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 4] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(16) Project [codegen id : 4] +(16) Project [codegen id : 1] Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] (17) ReusedExchange [Reuses operator id: 29] Output [2]: [d_date_sk#12, d_date#13] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] Keys [2]: [w_warehouse_name#7, i_item_id#11] Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] @@ -125,17 +125,17 @@ Results [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] Keys [2]: [w_warehouse_name#7, i_item_id#11] Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] Aggregate Attributes [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19] Results [4]: [w_warehouse_name#7, i_item_id#11, sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_before#20, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19 AS inv_after#21] -(24) Filter [codegen id : 5] +(24) Filter [codegen id : 2] Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] Condition : (CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) <= 1.5) END) @@ -147,7 +147,7 @@ Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRS Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_date#13] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt index 51fcfd010a..407648ea84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,20 +15,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/simplified.txt index 94925f8911..7c6d29a502 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) + WholeStageCodegen (2) Filter [inv_before,inv_after] HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] BroadcastHashJoin [inv_date_sk,d_date_sk] @@ -19,25 +19,19 @@ TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] InputAdapter ReusedExchange [d_date_sk,d_date] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/explain.txt index 5656916223..1053830b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/explain.txt @@ -142,7 +142,7 @@ Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_date#13] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt index ef02334507..097f5b20cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/simplified.txt index 1c2e80c991..9fc6fdc42b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/simplified.txt @@ -16,11 +16,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 CometFilter [w_warehouse_sk,w_warehouse_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 5656916223..1053830b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -142,7 +142,7 @@ Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_date#13] Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_date#13] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/extended.txt index ef02334507..097f5b20cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 1c2e80c991..9fc6fdc42b 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 @@ -16,11 +16,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 CometFilter [w_warehouse_sk,w_warehouse_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/explain.txt index f25f305679..bb531620cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (25) +- * HashAggregate (24) - +- * CometColumnarToRow (23) + +- CometNativeColumnarToRow (23) +- CometColumnarExchange (22) +- * HashAggregate (21) +- * Expand (20) @@ -16,12 +16,12 @@ TakeOrderedAndProject (25) : : : +- Scan parquet spark_catalog.default.inventory (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.warehouse (14) @@ -34,23 +34,23 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (4) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] @@ -69,20 +69,20 @@ Condition : isnotnull(i_item_sk#7) Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] (11) BroadcastExchange Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] @@ -97,28 +97,28 @@ ReadSchema: struct Input [1]: [w_warehouse_sk#16] Condition : isnotnull(w_warehouse_sk#16) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [w_warehouse_sk#16] (17) BroadcastExchange Input [1]: [w_warehouse_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#16] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] -(20) Expand [codegen id : 4] +(20) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] Arguments: [[inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#3, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] @@ -129,10 +129,10 @@ Results [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_gr Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] Arguments: hashpartitioning(i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 5] +(23) CometNativeColumnarToRow Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] Functions [1]: [avg(inv_quantity_on_hand#3)] @@ -147,7 +147,7 @@ Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.date_dim (26) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_mo Input [2]: [d_date_sk#6, d_month_seq#28] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt index 9b15a52cff..e1766e64d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -15,22 +15,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/simplified.txt index 092e187177..baff9284b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,26 +19,20 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + CometNativeColumnarToRow + CometFilter [w_warehouse_sk] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/explain.txt index 02cfd90b87..0486c4e213 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/explain.txt @@ -147,7 +147,7 @@ Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt index 7369619d2f..34e673af7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/simplified.txt index 9119ee749d..2cc520ff12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/simplified.txt @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt index 02cfd90b87..0486c4e213 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt @@ -147,7 +147,7 @@ Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/extended.txt index 7369619d2f..34e673af7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 9119ee749d..2cc520ff12 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 @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/explain.txt index 0170395540..2f07843de9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (69) +CometNativeColumnarToRow (69) +- CometHashAggregate (68) +- CometExchange (67) +- CometHashAggregate (66) @@ -18,7 +18,7 @@ : : : +- * Project (20) : : : +- * Filter (19) : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) + : : : +- CometNativeColumnarToRow (17) : : : +- CometColumnarExchange (16) : : : +- * HashAggregate (15) : : : +- * Project (14) @@ -30,7 +30,7 @@ : : : : : +- Scan parquet spark_catalog.default.store_sales (3) : : : : +- ReusedExchange (6) : : : +- BroadcastExchange (12) - : : : +- * CometColumnarToRow (11) + : : : +- CometNativeColumnarToRow (11) : : : +- CometFilter (10) : : : +- CometNativeScan parquet spark_catalog.default.item (9) : : +- CometSort (39) @@ -77,7 +77,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (3) Scan parquet spark_catalog.default.store_sales @@ -88,23 +88,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(4) ColumnarToRow [codegen id : 3] +(4) ColumnarToRow [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(5) Filter [codegen id : 3] +(5) Filter [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) (6) ReusedExchange [Reuses operator id: 79] Output [2]: [d_date_sk#10, d_date#11] -(7) BroadcastHashJoin [codegen id : 3] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(8) Project [codegen id : 3] +(8) Project [codegen id : 1] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] @@ -119,24 +119,24 @@ ReadSchema: struct Input [2]: [i_item_sk#12, i_item_desc#13] Condition : isnotnull(i_item_sk#12) -(11) CometColumnarToRow [codegen id : 2] +(11) CometNativeColumnarToRow Input [2]: [i_item_sk#12, i_item_desc#13] (12) BroadcastExchange Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 1] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 1] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] @@ -147,21 +147,21 @@ Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(19) Filter [codegen id : 4] +(19) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(20) Project [codegen id : 4] +(20) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] @@ -169,13 +169,13 @@ Input [2]: [item_sk#18, cnt#19] Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 5] +(22) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(23) Project [codegen id : 5] +(23) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -296,19 +296,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#6)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 10] +(49) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] (50) ReusedExchange [Reuses operator id: 21] Output [1]: [item_sk#18] -(51) BroadcastHashJoin [codegen id : 10] +(51) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#34] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(52) Project [codegen id : 10] +(52) Project [codegen id : 6] Output [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] @@ -379,14 +379,14 @@ Input [2]: [sum#41, isEmpty#42] Keys: [] Functions [1]: [sum(sales#33)] -(69) CometColumnarToRow [codegen id : 11] +(69) CometNativeColumnarToRow Input [1]: [sum(sales)#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometProject (72) +- CometFilter (71) +- CometNativeScan parquet spark_catalog.default.date_dim (70) @@ -407,7 +407,7 @@ Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2 Input [3]: [d_date_sk#30, d_year#31, d_moy#32] Arguments: [d_date_sk#30], [d_date_sk#30] -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [1]: [d_date_sk#30] (74) BroadcastExchange @@ -416,7 +416,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometFilter (76) +- CometNativeScan parquet spark_catalog.default.date_dim (75) @@ -437,7 +437,7 @@ Condition : (d_year#44 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#44] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (79) BroadcastExchange @@ -446,11 +446,11 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#28, [id=#29] * HashAggregate (99) -+- * CometColumnarToRow (98) ++- CometNativeColumnarToRow (98) +- CometColumnarExchange (97) +- * HashAggregate (96) +- * HashAggregate (95) - +- * CometColumnarToRow (94) + +- CometNativeColumnarToRow (94) +- CometColumnarExchange (93) +- * HashAggregate (92) +- * Project (91) @@ -461,7 +461,7 @@ Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquer : : +- * ColumnarToRow (81) : : +- Scan parquet spark_catalog.default.store_sales (80) : +- BroadcastExchange (86) - : +- * CometColumnarToRow (85) + : +- CometNativeColumnarToRow (85) : +- CometFilter (84) : +- CometNativeScan parquet spark_catalog.default.customer (83) +- ReusedExchange (89) @@ -475,10 +475,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#48), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(81) ColumnarToRow [codegen id : 3] +(81) ColumnarToRow [codegen id : 1] Input [4]: [ss_customer_sk#45, ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48] -(82) Filter [codegen id : 3] +(82) Filter [codegen id : 1] Input [4]: [ss_customer_sk#45, ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48] Condition : isnotnull(ss_customer_sk#45) @@ -493,37 +493,37 @@ ReadSchema: struct Input [1]: [c_customer_sk#50] Condition : isnotnull(c_customer_sk#50) -(85) CometColumnarToRow [codegen id : 1] +(85) CometNativeColumnarToRow Input [1]: [c_customer_sk#50] (86) BroadcastExchange Input [1]: [c_customer_sk#50] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -(87) BroadcastHashJoin [codegen id : 3] +(87) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#45] Right keys [1]: [c_customer_sk#50] Join type: Inner Join condition: None -(88) Project [codegen id : 3] +(88) Project [codegen id : 1] Output [4]: [ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48, c_customer_sk#50] Input [5]: [ss_customer_sk#45, ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48, c_customer_sk#50] (89) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#51] -(90) BroadcastHashJoin [codegen id : 3] +(90) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#48] Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(91) Project [codegen id : 3] +(91) Project [codegen id : 1] Output [3]: [ss_quantity#46, ss_sales_price#47, c_customer_sk#50] Input [5]: [ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48, c_customer_sk#50, d_date_sk#51] -(92) HashAggregate [codegen id : 3] +(92) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#46, ss_sales_price#47, c_customer_sk#50] Keys [1]: [c_customer_sk#50] Functions [1]: [partial_sum((cast(ss_quantity#46 as decimal(10,0)) * ss_sales_price#47))] @@ -534,17 +534,17 @@ Results [3]: [c_customer_sk#50, sum#54, isEmpty#55] Input [3]: [c_customer_sk#50, sum#54, isEmpty#55] Arguments: hashpartitioning(c_customer_sk#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(94) CometColumnarToRow [codegen id : 4] +(94) CometNativeColumnarToRow Input [3]: [c_customer_sk#50, sum#54, isEmpty#55] -(95) HashAggregate [codegen id : 4] +(95) HashAggregate [codegen id : 2] Input [3]: [c_customer_sk#50, sum#54, isEmpty#55] Keys [1]: [c_customer_sk#50] Functions [1]: [sum((cast(ss_quantity#46 as decimal(10,0)) * ss_sales_price#47))] Aggregate Attributes [1]: [sum((cast(ss_quantity#46 as decimal(10,0)) * ss_sales_price#47))#56] Results [1]: [sum((cast(ss_quantity#46 as decimal(10,0)) * ss_sales_price#47))#56 AS csales#57] -(96) HashAggregate [codegen id : 4] +(96) HashAggregate [codegen id : 2] Input [1]: [csales#57] Keys: [] Functions [1]: [partial_max(csales#57)] @@ -555,10 +555,10 @@ Results [1]: [max#59] Input [1]: [max#59] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(98) CometColumnarToRow [codegen id : 5] +(98) CometNativeColumnarToRow Input [1]: [max#59] -(99) HashAggregate [codegen id : 5] +(99) HashAggregate [codegen id : 3] Input [1]: [max#59] Keys: [] Functions [1]: [max(csales#57)] @@ -567,7 +567,7 @@ Results [1]: [max(csales#57)#60 AS tpcds_cmax#61] Subquery:4 Hosting operator id = 80 Hosting Expression = ss_sold_date_sk#48 IN dynamicpruning#49 BroadcastExchange (104) -+- * CometColumnarToRow (103) ++- CometNativeColumnarToRow (103) +- CometProject (102) +- CometFilter (101) +- CometNativeScan parquet spark_catalog.default.date_dim (100) @@ -588,7 +588,7 @@ Condition : (d_year#62 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#51)) Input [2]: [d_date_sk#51, d_year#62] Arguments: [d_date_sk#51], [d_date_sk#51] -(103) CometColumnarToRow [codegen id : 1] +(103) CometNativeColumnarToRow Input [1]: [d_date_sk#51] (104) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt index aadeb13f7b..7040e78da1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -23,7 +23,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -35,17 +35,17 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort @@ -53,11 +53,11 @@ CometColumnarToRow : : +- CometFilter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -69,16 +69,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -112,7 +112,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -124,17 +124,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/simplified.txt index d42a6ba29e..1a34e6b5b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/simplified.txt @@ -1,142 +1,130 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - 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] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] +CometNativeColumnarToRow + CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] + CometExchange #1 + 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] + CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + WholeStageCodegen (3) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (1) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (3) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_customer_sk] #11 + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - 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] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #15 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [d_date_sk] #14 + BroadcastExchange #13 + CometNativeColumnarToRow + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + ReusedExchange [d_date_sk] #12 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometExchange [c_customer_sk] #8 + 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] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #9 + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_quantity,ws_list_price] [sales] + CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] + CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #15 + WholeStageCodegen (6) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/explain.txt index bce5d2805d..97b304a7a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/explain.txt @@ -377,7 +377,7 @@ Input [1]: [sum(sales)#43] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 Input [3]: [d_date_sk#29, d_year#30, d_moy#31] Arguments: [d_date_sk#29], [d_date_sk#29] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -541,7 +541,7 @@ Input [1]: [tpcds_cmax#56] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 BroadcastExchange (100) -+- * CometColumnarToRow (99) ++- CometNativeColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) Input [2]: [d_date_sk#50, d_year#51] Arguments: [d_date_sk#50], [d_date_sk#50] -(99) CometColumnarToRow [codegen id : 1] +(99) CometNativeColumnarToRow Input [1]: [d_date_sk#50] (100) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt index a84226b077..0d7391c0d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,7 +112,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/simplified.txt index 8f1bddf6c0..38f04e8e26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/simplified.txt @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -36,12 +34,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt index bce5d2805d..97b304a7a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt @@ -377,7 +377,7 @@ Input [1]: [sum(sales)#43] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 Input [3]: [d_date_sk#29, d_year#30, d_moy#31] Arguments: [d_date_sk#29], [d_date_sk#29] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -541,7 +541,7 @@ Input [1]: [tpcds_cmax#56] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 BroadcastExchange (100) -+- * CometColumnarToRow (99) ++- CometNativeColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) Input [2]: [d_date_sk#50, d_year#51] Arguments: [d_date_sk#50], [d_date_sk#50] -(99) CometColumnarToRow [codegen id : 1] +(99) CometNativeColumnarToRow Input [1]: [d_date_sk#50] (100) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/extended.txt index a84226b077..0d7391c0d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,7 +112,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 8f1bddf6c0..38f04e8e26 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 @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -36,12 +34,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/explain.txt index c678959b08..e18e9fe9eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (90) +CometNativeColumnarToRow (90) +- CometTakeOrderedAndProject (89) +- CometUnion (88) :- CometHashAggregate (64) @@ -21,7 +21,7 @@ : : : : +- * Project (21) : : : : +- * Filter (20) : : : : +- * HashAggregate (19) - : : : : +- * CometColumnarToRow (18) + : : : : +- CometNativeColumnarToRow (18) : : : : +- CometColumnarExchange (17) : : : : +- * HashAggregate (16) : : : : +- * Project (15) @@ -33,7 +33,7 @@ : : : : : : +- Scan parquet spark_catalog.default.store_sales (4) : : : : : +- ReusedExchange (7) : : : : +- BroadcastExchange (13) - : : : : +- * CometColumnarToRow (12) + : : : : +- CometNativeColumnarToRow (12) : : : : +- CometFilter (11) : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : +- CometSort (40) @@ -99,10 +99,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 3] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Condition : isnotnull(cs_bill_customer_sk#1) @@ -114,23 +114,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 3] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -(6) Filter [codegen id : 3] +(6) Filter [codegen id : 1] Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] Condition : isnotnull(ss_item_sk#7) (7) ReusedExchange [Reuses operator id: 100] Output [2]: [d_date_sk#10, d_date#11] -(8) BroadcastHashJoin [codegen id : 3] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(9) Project [codegen id : 3] +(9) Project [codegen id : 1] Output [2]: [ss_item_sk#7, d_date#11] Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] @@ -145,24 +145,24 @@ ReadSchema: struct Input [2]: [i_item_sk#12, i_item_desc#13] Condition : isnotnull(i_item_sk#12) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [i_item_sk#12, i_item_desc#13] (13) BroadcastExchange Input [2]: [i_item_sk#12, i_item_desc#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(14) BroadcastHashJoin [codegen id : 3] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#7] Right keys [1]: [i_item_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 3] +(15) Project [codegen id : 1] Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] -(16) HashAggregate [codegen id : 3] +(16) HashAggregate [codegen id : 1] Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [partial_count(1)] @@ -173,21 +173,21 @@ Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(18) CometColumnarToRow [codegen id : 4] +(18) CometNativeColumnarToRow Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 2] Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#17] Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] -(20) Filter [codegen id : 4] +(20) Filter [codegen id : 2] Input [2]: [item_sk#18, cnt#19] Condition : (cnt#19 > 4) -(21) Project [codegen id : 4] +(21) Project [codegen id : 2] Output [1]: [item_sk#18] Input [2]: [item_sk#18, cnt#19] @@ -195,13 +195,13 @@ Input [2]: [item_sk#18, cnt#19] Input [1]: [item_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 5] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#2] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 5] +(24) Project [codegen id : 3] Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] @@ -394,23 +394,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(66) ColumnarToRow [codegen id : 10] +(66) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -(67) Filter [codegen id : 10] +(67) Filter [codegen id : 6] Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] Condition : isnotnull(ws_bill_customer_sk#41) (68) ReusedExchange [Reuses operator id: 22] Output [1]: [item_sk#18] -(69) BroadcastHashJoin [codegen id : 10] +(69) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#40] Right keys [1]: [item_sk#18] Join type: LeftSemi Join condition: None -(70) Project [codegen id : 10] +(70) Project [codegen id : 6] Output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] @@ -493,14 +493,14 @@ Child 1 Input [3]: [c_last_name#47, c_first_name#46, sales#52] Input [3]: [c_last_name#34, c_first_name#33, sales#51] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#34 ASC NULLS FIRST,c_first_name#33 ASC NULLS FIRST,sales#51 ASC NULLS FIRST], output=[c_last_name#34,c_first_name#33,sales#51]), [c_last_name#34, c_first_name#33, sales#51], 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, sales#51 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, sales#51] -(90) CometColumnarToRow [codegen id : 11] +(90) CometNativeColumnarToRow Input [3]: [c_last_name#34, c_first_name#33, sales#51] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (95) -+- * CometColumnarToRow (94) ++- CometNativeColumnarToRow (94) +- CometProject (93) +- CometFilter (92) +- CometNativeScan parquet spark_catalog.default.date_dim (91) @@ -521,7 +521,7 @@ Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2 Input [3]: [d_date_sk#35, d_year#36, d_moy#37] Arguments: [d_date_sk#35], [d_date_sk#35] -(94) CometColumnarToRow [codegen id : 1] +(94) CometNativeColumnarToRow Input [1]: [d_date_sk#35] (95) BroadcastExchange @@ -530,7 +530,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (100) -+- * CometColumnarToRow (99) ++- CometNativeColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometNativeScan parquet spark_catalog.default.date_dim (96) @@ -551,7 +551,7 @@ Condition : (d_year#53 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#53] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(99) CometColumnarToRow [codegen id : 1] +(99) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (100) BroadcastExchange @@ -560,11 +560,11 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#28, [id=#29] * HashAggregate (120) -+- * CometColumnarToRow (119) ++- CometNativeColumnarToRow (119) +- CometColumnarExchange (118) +- * HashAggregate (117) +- * HashAggregate (116) - +- * CometColumnarToRow (115) + +- CometNativeColumnarToRow (115) +- CometColumnarExchange (114) +- * HashAggregate (113) +- * Project (112) @@ -575,7 +575,7 @@ Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquer : : +- * ColumnarToRow (102) : : +- Scan parquet spark_catalog.default.store_sales (101) : +- BroadcastExchange (107) - : +- * CometColumnarToRow (106) + : +- CometNativeColumnarToRow (106) : +- CometFilter (105) : +- CometNativeScan parquet spark_catalog.default.customer (104) +- ReusedExchange (110) @@ -589,10 +589,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(102) ColumnarToRow [codegen id : 3] +(102) ColumnarToRow [codegen id : 1] Input [4]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57] -(103) Filter [codegen id : 3] +(103) Filter [codegen id : 1] Input [4]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_customer_sk#54) @@ -607,37 +607,37 @@ ReadSchema: struct Input [1]: [c_customer_sk#59] Condition : isnotnull(c_customer_sk#59) -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [c_customer_sk#59] (107) BroadcastExchange Input [1]: [c_customer_sk#59] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(108) BroadcastHashJoin [codegen id : 3] +(108) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#54] Right keys [1]: [c_customer_sk#59] Join type: Inner Join condition: None -(109) Project [codegen id : 3] +(109) Project [codegen id : 1] Output [4]: [ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57, c_customer_sk#59] Input [5]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57, c_customer_sk#59] (110) ReusedExchange [Reuses operator id: 125] Output [1]: [d_date_sk#60] -(111) BroadcastHashJoin [codegen id : 3] +(111) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#60] Join type: Inner Join condition: None -(112) Project [codegen id : 3] +(112) Project [codegen id : 1] Output [3]: [ss_quantity#55, ss_sales_price#56, c_customer_sk#59] Input [5]: [ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57, c_customer_sk#59, d_date_sk#60] -(113) HashAggregate [codegen id : 3] +(113) HashAggregate [codegen id : 1] Input [3]: [ss_quantity#55, ss_sales_price#56, c_customer_sk#59] Keys [1]: [c_customer_sk#59] Functions [1]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))] @@ -648,17 +648,17 @@ Results [3]: [c_customer_sk#59, sum#63, isEmpty#64] Input [3]: [c_customer_sk#59, sum#63, isEmpty#64] Arguments: hashpartitioning(c_customer_sk#59, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(115) CometColumnarToRow [codegen id : 4] +(115) CometNativeColumnarToRow Input [3]: [c_customer_sk#59, sum#63, isEmpty#64] -(116) HashAggregate [codegen id : 4] +(116) HashAggregate [codegen id : 2] Input [3]: [c_customer_sk#59, sum#63, isEmpty#64] Keys [1]: [c_customer_sk#59] Functions [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))] Aggregate Attributes [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))#65] Results [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))#65 AS csales#66] -(117) HashAggregate [codegen id : 4] +(117) HashAggregate [codegen id : 2] Input [1]: [csales#66] Keys: [] Functions [1]: [partial_max(csales#66)] @@ -669,10 +669,10 @@ Results [1]: [max#68] Input [1]: [max#68] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(119) CometColumnarToRow [codegen id : 5] +(119) CometNativeColumnarToRow Input [1]: [max#68] -(120) HashAggregate [codegen id : 5] +(120) HashAggregate [codegen id : 3] Input [1]: [max#68] Keys: [] Functions [1]: [max(csales#66)] @@ -681,7 +681,7 @@ Results [1]: [max(csales#66)#69 AS tpcds_cmax#70] Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 BroadcastExchange (125) -+- * CometColumnarToRow (124) ++- CometNativeColumnarToRow (124) +- CometProject (123) +- CometFilter (122) +- CometNativeScan parquet spark_catalog.default.date_dim (121) @@ -702,7 +702,7 @@ Condition : (d_year#71 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#60)) Input [2]: [d_date_sk#60, d_year#71] Arguments: [d_date_sk#60], [d_date_sk#60] -(124) CometColumnarToRow [codegen id : 1] +(124) CometNativeColumnarToRow Input [1]: [d_date_sk#60] (125) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt index d41bf0802a..188775e7df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometUnion :- CometHashAggregate @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -26,7 +26,7 @@ CometColumnarToRow : : : : +- Project : : : : +- Filter : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project @@ -38,17 +38,17 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- CometSort @@ -56,11 +56,11 @@ CometColumnarToRow : : : +- CometFilter : : : : +- Subquery : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project @@ -72,16 +72,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -154,17 +154,17 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/simplified.txt index 41f01311f8..d543da014a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/simplified.txt @@ -1,164 +1,152 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - 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 [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] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + CometTakeOrderedAndProject [c_last_name,c_first_name,sales] + CometUnion [c_last_name,c_first_name,sales] + 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 [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] + CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometColumnarExchange [cs_bill_customer_sk] #2 + WholeStageCodegen (3) + Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + BroadcastHashJoin [cs_item_sk,item_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (1) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + Subquery #3 + WholeStageCodegen (3) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [csales] [max,max] + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_customer_sk] #11 + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - 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] - CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #14 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #15 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - CometBroadcastExchange [d_date_sk] #16 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] #17 - 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] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #18 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 + BroadcastExchange #13 + CometNativeColumnarToRow + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + ReusedExchange [d_date_sk] #12 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometExchange [c_customer_sk] #8 + 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] + CometFilter [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [c_customer_sk] #9 + CometFilter [c_customer_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #14 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] + CometSort [c_customer_sk,c_first_name,c_last_name] + CometExchange [c_customer_sk] #15 + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] CometSort [c_customer_sk] CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 - ReusedExchange [d_date_sk] #16 + CometBroadcastExchange [d_date_sk] #16 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + 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] #17 + 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] + CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] + CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometColumnarExchange [ws_bill_customer_sk] #18 + WholeStageCodegen (6) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + CometSort [c_customer_sk] + CometProject [c_customer_sk] + CometFilter [c_customer_sk,ssales] + ReusedSubquery [tpcds_cmax] #3 + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 + ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/explain.txt index 67e4e39057..154a97e7cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/explain.txt @@ -491,7 +491,7 @@ Input [3]: [c_last_name#33, c_first_name#32, sales#51] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (93) -+- * CometColumnarToRow (92) ++- CometNativeColumnarToRow (92) +- CometProject (91) +- CometFilter (90) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) @@ -512,7 +512,7 @@ Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2 Input [3]: [d_date_sk#34, d_year#35, d_moy#36] Arguments: [d_date_sk#34], [d_date_sk#34] -(92) CometColumnarToRow [codegen id : 1] +(92) CometNativeColumnarToRow Input [1]: [d_date_sk#34] (93) BroadcastExchange @@ -521,7 +521,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometProject (96) +- CometFilter (95) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) @@ -542,7 +542,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (98) BroadcastExchange @@ -655,7 +655,7 @@ Input [1]: [tpcds_cmax#65] Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#56 IN dynamicpruning#57 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -676,7 +676,7 @@ Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#59)) Input [2]: [d_date_sk#59, d_year#60] Arguments: [d_date_sk#59], [d_date_sk#59] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#59] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt index cd91f5de49..4838389d3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,7 +35,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/simplified.txt index 92563114ae..b81ee0b822 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -39,12 +37,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -73,12 +69,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 67e4e39057..154a97e7cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -491,7 +491,7 @@ Input [3]: [c_last_name#33, c_first_name#32, sales#51] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (93) -+- * CometColumnarToRow (92) ++- CometNativeColumnarToRow (92) +- CometProject (91) +- CometFilter (90) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) @@ -512,7 +512,7 @@ Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2 Input [3]: [d_date_sk#34, d_year#35, d_moy#36] Arguments: [d_date_sk#34], [d_date_sk#34] -(92) CometColumnarToRow [codegen id : 1] +(92) CometNativeColumnarToRow Input [1]: [d_date_sk#34] (93) BroadcastExchange @@ -521,7 +521,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometProject (96) +- CometFilter (95) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) @@ -542,7 +542,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [2]: [d_date_sk#10, d_date#11] (98) BroadcastExchange @@ -655,7 +655,7 @@ Input [1]: [tpcds_cmax#65] Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#56 IN dynamicpruning#57 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -676,7 +676,7 @@ Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#59)) Input [2]: [d_date_sk#59, d_year#60] Arguments: [d_date_sk#59], [d_date_sk#59] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#59] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/extended.txt index cd91f5de49..4838389d3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,7 +35,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 92563114ae..b81ee0b822 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] @@ -39,12 +37,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -73,12 +69,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] ReusedExchange [c_customer_sk] #10 CometBroadcastExchange [d_date_sk] #14 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/explain.txt index acc9219af4..683a6d0f59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == * Filter (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) + :- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometProject (24) @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometNativeColumnarToRow Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometNativeScan parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometNativeColumnarToRow Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -272,16 +272,16 @@ Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subqu Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometColumnarExchange (75) +- * HashAggregate (74) +- * HashAggregate (73) - +- * CometColumnarToRow (72) + +- CometNativeColumnarToRow (72) +- CometColumnarExchange (71) +- * HashAggregate (70) +- * Project (69) +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) + :- CometNativeColumnarToRow (66) : +- CometProject (65) : +- CometBroadcastHashJoin (64) : :- CometProject (62) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometNativeColumnarToRow Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] @@ -426,10 +426,10 @@ Results [2]: [sum#71, count#72] Input [2]: [sum#71, count#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometNativeColumnarToRow Input [2]: [sum#71, count#72] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#71, count#72] Keys: [] Functions [1]: [avg(netpaid#40)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt index b65f56f327..03e6809902 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt @@ -1,16 +1,16 @@ Filter : +- Subquery : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow +: :- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometProject @@ -42,21 +42,21 @@ Filter : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometProject @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/simplified.txt index af8d5ee7aa..affed709e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -39,21 +39,21 @@ WholeStageCodegen (4) InputAdapter ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/explain.txt index e478cdfc01..09a06850cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] @@ -426,10 +426,10 @@ Results [2]: [sum#71, count#72] Input [2]: [sum#71, count#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#71, count#72] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#71, count#72] Keys: [] Functions [1]: [avg(netpaid#40)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt index e478cdfc01..09a06850cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] @@ -426,10 +426,10 @@ Results [2]: [sum#71, count#72] Input [2]: [sum#71, count#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#71, count#72] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#71, count#72] Keys: [] Functions [1]: [avg(netpaid#40)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/explain.txt index dbebe8f96d..0ee29d819f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/explain.txt @@ -1,16 +1,16 @@ == Physical Plan == * Filter (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) + :- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometProject (24) @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometNativeColumnarToRow Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometNativeScan parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometNativeColumnarToRow Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -272,16 +272,16 @@ Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subqu Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] * HashAggregate (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometColumnarExchange (75) +- * HashAggregate (74) +- * HashAggregate (73) - +- * CometColumnarToRow (72) + +- CometNativeColumnarToRow (72) +- CometColumnarExchange (71) +- * HashAggregate (70) +- * Project (69) +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) + :- CometNativeColumnarToRow (66) : +- CometProject (65) : +- CometBroadcastHashJoin (64) : :- CometProject (62) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometNativeColumnarToRow Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometNativeColumnarToRow Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] @@ -426,10 +426,10 @@ Results [2]: [sum#71, count#72] Input [2]: [sum#71, count#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometNativeColumnarToRow Input [2]: [sum#71, count#72] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#71, count#72] Keys: [] Functions [1]: [avg(netpaid#40)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt index b65f56f327..03e6809902 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt @@ -1,16 +1,16 @@ Filter : +- Subquery : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow +: :- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometProject @@ -42,21 +42,21 @@ Filter : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometProject @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/simplified.txt index af8d5ee7aa..affed709e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -39,21 +39,21 @@ WholeStageCodegen (4) InputAdapter ReusedExchange [ca_state,ca_zip,ca_country] #8 HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/explain.txt index 548959be1b..3a0adb25af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] @@ -426,10 +426,10 @@ Results [2]: [sum#71, count#72] Input [2]: [sum#71, count#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#71, count#72] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#71, count#72] Keys: [] Functions [1]: [avg(netpaid#40)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index 548959be1b..3a0adb25af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -42,7 +42,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -187,7 +187,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -205,24 +205,24 @@ Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apach Input [3]: [ca_state#32, ca_zip#33, ca_country#34] Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [3]: [ca_state#35, ca_zip#36, ca_country#34] (36) BroadcastExchange Input [3]: [ca_state#35, ca_zip#36, ca_country#34] Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#29, s_zip#16] Right keys [2]: [upper(ca_country#34), ca_zip#36] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -233,17 +233,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [partial_sum(netpaid#40)] @@ -254,17 +254,17 @@ Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty# Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] Functions [1]: [sum(netpaid#40)] Aggregate Attributes [1]: [sum(netpaid#40)#45] Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) @@ -378,23 +378,23 @@ Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] -(66) CometColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [codegen id : 1] Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] (67) ReusedExchange [Reuses operator id: 36] Output [3]: [ca_state#35, ca_zip#36, ca_country#66] -(68) BroadcastHashJoin [codegen id : 2] +(68) BroadcastHashJoin [codegen id : 1] Left keys [2]: [c_birth_country#65, s_zip#16] Right keys [2]: [upper(ca_country#66), ca_zip#36] Join type: Inner Join condition: None -(69) Project [codegen id : 2] +(69) Project [codegen id : 1] Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] -(70) HashAggregate [codegen id : 2] +(70) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] @@ -405,17 +405,17 @@ Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_ Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(72) CometColumnarToRow [codegen id : 3] +(72) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -(73) HashAggregate [codegen id : 3] +(73) HashAggregate [codegen id : 2] Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] -(74) HashAggregate [codegen id : 3] +(74) HashAggregate [codegen id : 2] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] @@ -426,10 +426,10 @@ Results [2]: [sum#71, count#72] Input [2]: [sum#71, count#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) CometColumnarToRow [codegen id : 4] +(76) CometColumnarToRow [codegen id : 3] Input [2]: [sum#71, count#72] -(77) HashAggregate [codegen id : 4] +(77) HashAggregate [codegen id : 3] Input [2]: [sum#71, count#72] Keys: [] Functions [1]: [avg(netpaid#40)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt index 8777e81a37..12050b3ad1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt @@ -42,7 +42,7 @@ Filter : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -88,7 +88,7 @@ Filter : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index 440f477d0f..fa1bbe0b3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #9 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -42,13 +42,13 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] @@ -86,9 +86,7 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] + CometFilter [ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/explain.txt index 15bdd29f61..17e2a46a14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- ReusedExchange (19) : : +- ReusedExchange (22) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.store (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.item (32) @@ -52,10 +52,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) @@ -78,13 +78,13 @@ Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnot Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 8] +(8) BroadcastHashJoin [codegen id : 3] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] Join type: Inner Join condition: None -(9) Project [codegen id : 8] +(9) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] @@ -107,52 +107,52 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 3] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] Join type: Inner Join condition: None -(15) Project [codegen id : 8] +(15) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] (16) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#18] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(18) Project [codegen id : 3] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#18] (19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#19] -(20) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_returned_date_sk#12] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(21) Project [codegen id : 8] +(21) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] (22) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#20] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(24) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] @@ -171,20 +171,20 @@ Condition : isnotnull(s_store_sk#21) Input [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] Arguments: [s_store_sk#21, s_store_id#24, s_store_name#23], [s_store_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#22, 16, true, false, true) AS s_store_id#24, s_store_name#23] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] (29) BroadcastExchange Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#21, s_store_id#24, s_store_name#23] @@ -203,24 +203,24 @@ Condition : isnotnull(i_item_sk#25) Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] Arguments: [i_item_sk#25, i_item_id#28, i_item_desc#27], [i_item_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#26, 16, true, false, true) AS i_item_id#28, i_item_desc#27] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] (36) BroadcastExchange Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#25] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(38) Project [codegen id : 3] Output [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_sk#25, i_item_id#28, i_item_desc#27] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 3] Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] @@ -231,10 +231,10 @@ Results [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum# Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] Arguments: hashpartitioning(i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 4] Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] @@ -249,7 +249,7 @@ Arguments: 100, [i_item_id#28 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 4) Input [3]: [d_date_sk#18, d_year#41, d_moy#42] Arguments: [d_date_sk#18], [d_date_sk#18] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#18] (48) BroadcastExchange @@ -279,7 +279,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -300,7 +300,7 @@ Condition : (((((isnotnull(d_moy#44) AND isnotnull(d_year#43)) AND (d_moy#44 >= Input [3]: [d_date_sk#19, d_year#43, d_moy#44] Arguments: [d_date_sk#19], [d_date_sk#19] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt index 26e924fd7e..15c6bec5dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,27 +42,27 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/simplified.txt index 288561a740..cc47e03799 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,12 +39,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -63,17 +59,13 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk,i_item_id,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/explain.txt index 6a9f9094d3..06156ea810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/explain.txt @@ -256,7 +256,7 @@ Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/simplified.txt index dcad304452..7d8d4694e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/simplified.txt @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 6a9f9094d3..06156ea810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -256,7 +256,7 @@ Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -277,7 +277,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (49) BroadcastExchange @@ -286,7 +286,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) @@ -307,7 +307,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/extended.txt index 4e1a4f3ec0..21afc9cb01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 dcad304452..7d8d4694e8 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 @@ -23,23 +23,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/explain.txt index fb4c487ea9..069d2d52eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * HashAggregate (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * Project (27) @@ -16,18 +16,18 @@ TakeOrderedAndProject (32) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : +- ReusedExchange (11) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.item (14) +- BroadcastExchange (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometNativeScan parquet spark_catalog.default.promotion (21) @@ -41,10 +41,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) @@ -63,33 +63,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#1] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] @@ -108,20 +108,20 @@ Condition : isnotnull(i_item_sk#15) Input [2]: [i_item_sk#15, i_item_id#16] Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#16, 16, true, false, true) AS i_item_id#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [i_item_sk#15, i_item_id#17] (18) BroadcastExchange Input [2]: [i_item_sk#15, i_item_id#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#2] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#15, i_item_id#17] @@ -140,24 +140,24 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Arguments: [p_promo_sk#18], [p_promo_sk#18] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [1]: [p_promo_sk#18] (25) BroadcastExchange Input [1]: [p_promo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_promo_sk#3] Right keys [1]: [p_promo_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] @@ -168,10 +168,10 @@ Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34 Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Keys [1]: [i_item_id#17] Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] @@ -186,7 +186,7 @@ Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#45] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt index 3b9d40795e..ceba8f5828 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -16,27 +16,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/simplified.txt index 7eb9c67611..23b7e6a8f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] BroadcastHashJoin [cs_promo_sk,p_promo_sk] @@ -20,35 +20,27 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometNativeColumnarToRow + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/explain.txt index 3d70460689..ae35309ada 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt index dbea7afe34..ea785136aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/simplified.txt index 0b56a47547..f040250da3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/simplified.txt @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index 3d70460689..ae35309ada 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/extended.txt index dbea7afe34..ea785136aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0b56a47547..f040250da3 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 @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/explain.txt index b9e3e82684..525997db60 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (33) +- * HashAggregate (32) - +- * CometColumnarToRow (31) + +- CometNativeColumnarToRow (31) +- CometColumnarExchange (30) +- * HashAggregate (29) +- * Expand (28) @@ -17,18 +17,18 @@ TakeOrderedAndProject (33) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : +- ReusedExchange (11) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.store (14) +- BroadcastExchange (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometNativeScan parquet spark_catalog.default.item (21) @@ -42,10 +42,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) @@ -64,33 +64,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 38] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -109,20 +109,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#15, s_state#16] Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [s_store_sk#15, s_state#17] (18) BroadcastExchange Input [2]: [s_store_sk#15, s_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] @@ -141,28 +141,28 @@ Condition : isnotnull(i_item_sk#18) Input [2]: [i_item_sk#18, i_item_id#19] Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [2]: [i_item_sk#18, i_item_id#20] (25) BroadcastExchange Input [2]: [i_item_sk#18, i_item_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] -(28) Expand [codegen id : 5] +(28) Expand [codegen id : 1] Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] -(29) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 1] Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] @@ -173,10 +173,10 @@ Results [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] Arguments: hashpartitioning(i_item_id#21, s_state#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(31) CometColumnarToRow [codegen id : 6] +(31) CometNativeColumnarToRow Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 2] Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] @@ -191,7 +191,7 @@ Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#49] Arguments: [d_date_sk#14], [d_date_sk#14] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt index a4946e4771..20f7517d3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -17,27 +17,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/simplified.txt index dcb4953707..452cc03fb3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] @@ -21,35 +21,27 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/explain.txt index eb158b2889..87c8eb2bc3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/explain.txt @@ -186,7 +186,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt index f550f3855c..bbfc91ed08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/simplified.txt index bdae0cc477..98686d7452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index eb158b2889..87c8eb2bc3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -186,7 +186,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/extended.txt index f550f3855c..bbfc91ed08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 bdae0cc477..98686d7452 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/explain.txt index 985274408b..bb87d4cc2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/explain.txt @@ -4,72 +4,72 @@ : :- * BroadcastNestedLoopJoin Inner BuildRight (50) : : :- * BroadcastNestedLoopJoin Inner BuildRight (37) : : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) +: : : : :- CometNativeColumnarToRow (11) : : : : : +- CometHashAggregate (10) : : : : : +- CometColumnarExchange (9) : : : : : +- * HashAggregate (8) : : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) +: : : : : +- CometNativeColumnarToRow (6) : : : : : +- CometExchange (5) : : : : : +- CometHashAggregate (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) : : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) +: : : : +- CometNativeColumnarToRow (22) : : : : +- CometHashAggregate (21) : : : : +- CometColumnarExchange (20) : : : : +- * HashAggregate (19) : : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) +: : : : +- CometNativeColumnarToRow (17) : : : : +- CometExchange (16) : : : : +- CometHashAggregate (15) : : : : +- CometProject (14) : : : : +- CometFilter (13) : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (12) : : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) +: : : +- CometNativeColumnarToRow (35) : : : +- CometHashAggregate (34) : : : +- CometColumnarExchange (33) : : : +- * HashAggregate (32) : : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) +: : : +- CometNativeColumnarToRow (30) : : : +- CometExchange (29) : : : +- CometHashAggregate (28) : : : +- CometProject (27) : : : +- CometFilter (26) : : : +- CometNativeScan parquet spark_catalog.default.store_sales (25) : : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) +: : +- CometNativeColumnarToRow (48) : : +- CometHashAggregate (47) : : +- CometColumnarExchange (46) : : +- * HashAggregate (45) : : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) +: : +- CometNativeColumnarToRow (43) : : +- CometExchange (42) : : +- CometHashAggregate (41) : : +- CometProject (40) : : +- CometFilter (39) : : +- CometNativeScan parquet spark_catalog.default.store_sales (38) : +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) +: +- CometNativeColumnarToRow (61) : +- CometHashAggregate (60) : +- CometColumnarExchange (59) : +- * HashAggregate (58) : +- * HashAggregate (57) -: +- * CometColumnarToRow (56) +: +- CometNativeColumnarToRow (56) : +- CometExchange (55) : +- CometHashAggregate (54) : +- CometProject (53) : +- CometFilter (52) : +- CometNativeScan parquet spark_catalog.default.store_sales (51) +- BroadcastExchange (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometHashAggregate (73) +- CometColumnarExchange (72) +- * HashAggregate (71) +- * HashAggregate (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometExchange (68) +- CometHashAggregate (67) +- CometProject (66) @@ -101,7 +101,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_li Input [4]: [ss_list_price#3, sum#6, count#7, count#8] Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [4]: [ss_list_price#3, sum#6, count#7, count#8] (7) HashAggregate [codegen id : 1] @@ -127,7 +127,7 @@ Input [4]: [sum#6, count#7, count#8, count#12] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -(11) CometColumnarToRow [codegen id : 12] +(11) CometNativeColumnarToRow Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] (12) CometNativeScan parquet spark_catalog.default.store_sales @@ -154,7 +154,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_l Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(17) CometColumnarToRow [codegen id : 2] +(17) CometNativeColumnarToRow Input [4]: [ss_list_price#18, sum#21, count#22, count#23] (18) HashAggregate [codegen id : 2] @@ -180,14 +180,14 @@ Input [4]: [sum#21, count#22, count#23, count#27] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] -(22) CometColumnarToRow [codegen id : 3] +(22) CometNativeColumnarToRow Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] (23) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(24) BroadcastNestedLoopJoin [codegen id : 12] +(24) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -215,17 +215,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_l Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(30) CometColumnarToRow [codegen id : 4] +(30) CometNativeColumnarToRow Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -(31) HashAggregate [codegen id : 4] +(31) HashAggregate [codegen id : 3] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys [1]: [ss_list_price#33] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] Results [4]: [ss_list_price#33, sum#36, count#37, count#38] -(32) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 3] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] @@ -241,14 +241,14 @@ Input [4]: [sum#36, count#37, count#38, count#42] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] -(35) CometColumnarToRow [codegen id : 5] +(35) CometNativeColumnarToRow Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] (36) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(37) BroadcastNestedLoopJoin [codegen id : 12] +(37) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -276,17 +276,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_l Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(43) CometColumnarToRow [codegen id : 6] +(43) CometNativeColumnarToRow Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -(44) HashAggregate [codegen id : 6] +(44) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys [1]: [ss_list_price#48] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] Results [4]: [ss_list_price#48, sum#51, count#52, count#53] -(45) HashAggregate [codegen id : 6] +(45) HashAggregate [codegen id : 4] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] @@ -302,14 +302,14 @@ Input [4]: [sum#51, count#52, count#53, count#57] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] -(48) CometColumnarToRow [codegen id : 7] +(48) CometNativeColumnarToRow Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] (49) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(50) BroadcastNestedLoopJoin [codegen id : 12] +(50) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -337,17 +337,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_l Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(56) CometColumnarToRow [codegen id : 8] +(56) CometNativeColumnarToRow Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -(57) HashAggregate [codegen id : 8] +(57) HashAggregate [codegen id : 5] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys [1]: [ss_list_price#63] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] Results [4]: [ss_list_price#63, sum#66, count#67, count#68] -(58) HashAggregate [codegen id : 8] +(58) HashAggregate [codegen id : 5] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] @@ -363,14 +363,14 @@ Input [4]: [sum#66, count#67, count#68, count#72] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] -(61) CometColumnarToRow [codegen id : 9] +(61) CometNativeColumnarToRow Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] (62) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(63) BroadcastNestedLoopJoin [codegen id : 12] +(63) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None @@ -398,17 +398,17 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_l Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(69) CometColumnarToRow [codegen id : 10] +(69) CometNativeColumnarToRow Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -(70) HashAggregate [codegen id : 10] +(70) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys [1]: [ss_list_price#78] Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] Results [4]: [ss_list_price#78, sum#81, count#82, count#83] -(71) HashAggregate [codegen id : 10] +(71) HashAggregate [codegen id : 6] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Keys: [] Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] @@ -424,14 +424,14 @@ Input [4]: [sum#81, count#82, count#83, count#87] Keys: [] Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] -(74) CometColumnarToRow [codegen id : 11] +(74) CometNativeColumnarToRow Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] (75) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(76) BroadcastNestedLoopJoin [codegen id : 12] +(76) BroadcastNestedLoopJoin [codegen id : 7] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/extended.txt index 37c297f68a..d05989eb4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/extended.txt @@ -3,72 +3,72 @@ BroadcastNestedLoopJoin : :- BroadcastNestedLoopJoin : : :- BroadcastNestedLoopJoin : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : :- CometColumnarToRow +: : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate : : : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : : +- CometColumnarToRow +: : : : : +- CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometHashAggregate : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- BroadcastExchange -: : : : +- CometColumnarToRow +: : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : +- CometColumnarToRow +: : : : +- CometNativeColumnarToRow : : : : +- CometExchange : : : : +- CometHashAggregate : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- BroadcastExchange -: : : +- CometColumnarToRow +: : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : +- CometColumnarToRow +: : : +- CometNativeColumnarToRow : : : +- CometExchange : : : +- CometHashAggregate : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- BroadcastExchange -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometExchange : : +- CometHashAggregate : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometExchange : +- CometHashAggregate : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometExchange +- CometHashAggregate +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/simplified.txt index a8540a4abb..96a591d9ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (12) +WholeStageCodegen (7) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] CometColumnarExchange #1 WholeStageCodegen (1) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometExchange [ss_list_price] #2 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] @@ -20,86 +20,76 @@ WholeStageCodegen (12) CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] InputAdapter BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #4 + WholeStageCodegen (2) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #5 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #7 + WholeStageCodegen (3) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #8 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) + InputAdapter + BroadcastExchange #9 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #10 + WholeStageCodegen (4) HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #11 CometHashAggregate [ss_list_price,sum,count,count] CometProject [ss_list_price] CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #12 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #13 + WholeStageCodegen (5) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #14 + CometHashAggregate [ss_list_price,sum,count,count] + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #15 + CometNativeColumnarToRow + CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] + CometColumnarExchange #16 + WholeStageCodegen (6) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + CometNativeColumnarToRow + CometExchange [ss_list_price] #17 + CometHashAggregate [ss_list_price,sum,count,count] + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/explain.txt index 20f1508413..43df55e471 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- ReusedExchange (19) : : +- ReusedExchange (22) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.store (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.item (32) @@ -52,10 +52,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 3] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) @@ -78,13 +78,13 @@ Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnot Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 8] +(8) BroadcastHashJoin [codegen id : 3] Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] Join type: Inner Join condition: None -(9) Project [codegen id : 8] +(9) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] @@ -107,52 +107,52 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 8] +(14) BroadcastHashJoin [codegen id : 3] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] Join type: Inner Join condition: None -(15) Project [codegen id : 8] +(15) Project [codegen id : 3] Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] (16) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#19] -(17) BroadcastHashJoin [codegen id : 8] +(17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(18) Project [codegen id : 8] +(18) Project [codegen id : 3] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] (19) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#20] -(20) BroadcastHashJoin [codegen id : 8] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sr_returned_date_sk#12] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(21) Project [codegen id : 8] +(21) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] (22) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#21] -(23) BroadcastHashJoin [codegen id : 8] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#17] Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None -(24) Project [codegen id : 8] +(24) Project [codegen id : 3] Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] @@ -171,20 +171,20 @@ Condition : isnotnull(s_store_sk#22) Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] Arguments: [s_store_sk#22, s_store_id#25, s_store_name#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#23, 16, true, false, true) AS s_store_id#25, s_store_name#24] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] (29) BroadcastExchange Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 8] +(30) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#22] Join type: Inner Join condition: None -(31) Project [codegen id : 8] +(31) Project [codegen id : 3] Output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_store_id#25, s_store_name#24] @@ -203,24 +203,24 @@ Condition : isnotnull(i_item_sk#26) Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] Arguments: [i_item_sk#26, i_item_id#29, i_item_desc#28], [i_item_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#27, 16, true, false, true) AS i_item_id#29, i_item_desc#28] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] (36) BroadcastExchange Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(38) Project [codegen id : 8] +(38) Project [codegen id : 3] Output [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_sk#26, i_item_id#29, i_item_desc#28] -(39) HashAggregate [codegen id : 8] +(39) HashAggregate [codegen id : 3] Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] @@ -231,10 +231,10 @@ Results [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum# Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] Arguments: hashpartitioning(i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 9] +(41) CometNativeColumnarToRow Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 4] Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] @@ -249,7 +249,7 @@ Arguments: 100, [i_item_id#29 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -270,7 +270,7 @@ Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 9) Input [3]: [d_date_sk#19, d_year#42, d_moy#43] Arguments: [d_date_sk#19], [d_date_sk#19] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (48) BroadcastExchange @@ -279,7 +279,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -300,7 +300,7 @@ Condition : (((((isnotnull(d_moy#45) AND isnotnull(d_year#44)) AND (d_moy#45 >= Input [3]: [d_date_sk#20, d_year#44, d_moy#45] Arguments: [d_date_sk#20], [d_date_sk#20] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#20] (53) BroadcastExchange @@ -309,7 +309,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (58) -+- * CometColumnarToRow (57) ++- CometNativeColumnarToRow (57) +- CometProject (56) +- CometFilter (55) +- CometNativeScan parquet spark_catalog.default.date_dim (54) @@ -330,7 +330,7 @@ Condition : (d_year#46 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) Input [2]: [d_date_sk#21, d_year#46] Arguments: [d_date_sk#21], [d_date_sk#21] -(57) CometColumnarToRow [codegen id : 1] +(57) CometNativeColumnarToRow Input [1]: [d_date_sk#21] (58) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt index dacaab515f..bd9fa52aa7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,32 +42,32 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/simplified.txt index 54bdbeeb5a..19ffcce191 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) + WholeStageCodegen (3) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) @@ -41,12 +39,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) @@ -56,12 +52,10 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter @@ -70,17 +64,13 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] InputAdapter BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] + CometFilter [i_item_sk,i_item_id,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/explain.txt index 88c51b72ee..58860a2993 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/explain.txt @@ -275,7 +275,7 @@ Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -296,7 +296,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (52) BroadcastExchange @@ -305,7 +305,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -326,7 +326,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (57) BroadcastExchange @@ -335,7 +335,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometFilter (59) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) @@ -356,7 +356,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#25] (62) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt index fe03fef8a7..8c47903ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/simplified.txt index 15b992d3a4..e772ae6f33 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/simplified.txt @@ -23,34 +23,28 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index 88c51b72ee..58860a2993 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt @@ -275,7 +275,7 @@ Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -296,7 +296,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) Input [3]: [d_date_sk#19, d_year#20, d_moy#21] Arguments: [d_date_sk#19], [d_date_sk#19] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (52) BroadcastExchange @@ -305,7 +305,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -326,7 +326,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= Input [3]: [d_date_sk#22, d_year#23, d_moy#24] Arguments: [d_date_sk#22], [d_date_sk#22] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (57) BroadcastExchange @@ -335,7 +335,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (62) -+- * CometColumnarToRow (61) ++- CometNativeColumnarToRow (61) +- CometProject (60) +- CometFilter (59) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) @@ -356,7 +356,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(61) CometColumnarToRow [codegen id : 1] +(61) CometNativeColumnarToRow Input [1]: [d_date_sk#25] (62) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/extended.txt index fe03fef8a7..8c47903ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -30,7 +30,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 15b992d3a4..e772ae6f33 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 @@ -23,34 +23,28 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] #3 CometFilter [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] #5 CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #7 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/explain.txt index 9a5ab78af8..775ad33d71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/simplified.txt index c19e6caf52..0a6cce1eb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - 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 [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] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] + 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 [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] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/explain.txt index 7992727538..ee4e4596fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (54) : : +- * BroadcastHashJoin Inner BuildRight (38) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) @@ -20,18 +20,18 @@ TakeOrderedAndProject (54) : : : : : +- Scan parquet spark_catalog.default.web_returns (1) : : : : +- ReusedExchange (4) : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) + : : : +- CometNativeColumnarToRow (10) : : : +- CometProject (9) : : : +- CometFilter (8) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : : +- BroadcastExchange (37) : : +- * Filter (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) + : : +- CometNativeColumnarToRow (30) : : +- CometColumnarExchange (29) : : +- * HashAggregate (28) : : +- * Project (27) @@ -44,12 +44,12 @@ TakeOrderedAndProject (54) : : : +- ReusedExchange (22) : : +- ReusedExchange (25) : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometProject (42) : +- CometFilter (41) : +- CometNativeScan parquet spark_catalog.default.customer (40) +- BroadcastExchange (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometNativeScan parquet spark_catalog.default.customer_address (47) @@ -63,23 +63,23 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [wr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] @@ -98,24 +98,24 @@ Condition : (isnotnull(ca_address_sk#7) AND isnotnull(staticinvoke(class org.apa Input [2]: [ca_address_sk#7, ca_state#8] Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#9] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [ca_address_sk#7, ca_state#9] (11) BroadcastExchange Input [2]: [ca_address_sk#7, ca_state#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [wr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#9] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] @@ -126,17 +126,17 @@ Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 11] +(16) CometNativeColumnarToRow Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -(17) HashAggregate [codegen id : 11] +(17) HashAggregate [codegen id : 5] Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] -(18) Filter [codegen id : 11] +(18) Filter [codegen id : 5] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) @@ -148,40 +148,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 2] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -(21) Filter [codegen id : 6] +(21) Filter [codegen id : 2] Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] Condition : isnotnull(wr_returning_addr_sk#2) (22) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(24) Project [codegen id : 2] Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] (25) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#7, ca_state#9] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 2] Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#9] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] @@ -192,17 +192,17 @@ Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#17] Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#17] Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#17] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#17] Keys [2]: [wr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] Results [2]: [ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] -(32) HashAggregate [codegen id : 7] +(32) HashAggregate [codegen id : 3] Input [2]: [ctr_state#14, ctr_total_return#15] Keys [1]: [ctr_state#14] Functions [1]: [partial_avg(ctr_total_return#15)] @@ -213,17 +213,17 @@ Results [3]: [ctr_state#14, sum#20, count#21] Input [3]: [ctr_state#14, sum#20, count#21] Arguments: hashpartitioning(ctr_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 8] +(34) CometNativeColumnarToRow Input [3]: [ctr_state#14, sum#20, count#21] -(35) HashAggregate [codegen id : 8] +(35) HashAggregate [codegen id : 4] Input [3]: [ctr_state#14, sum#20, count#21] Keys [1]: [ctr_state#14] Functions [1]: [avg(ctr_total_return#15)] Aggregate Attributes [1]: [avg(ctr_total_return#15)#22] Results [2]: [(avg(ctr_total_return#15)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#14 AS ctr_state#14#24] -(36) Filter [codegen id : 8] +(36) Filter [codegen id : 4] Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) @@ -231,13 +231,13 @@ Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 11] +(38) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_state#14] Right keys [1]: [ctr_state#14#24] Join type: Inner Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) -(39) Project [codegen id : 11] +(39) Project [codegen id : 5] Output [2]: [ctr_customer_sk#13, ctr_total_return#15] Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] @@ -256,20 +256,20 @@ Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] Arguments: [c_customer_sk#25, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#39, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#28, 10, true, false, true) AS c_salutation#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#36, 13, true, false, true) AS c_login#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#37, 50, true, false, true) AS c_email_address#45, c_last_review_date#38] -(43) CometColumnarToRow [codegen id : 9] +(43) CometNativeColumnarToRow Input [14]: [c_customer_sk#25, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38] (44) BroadcastExchange Input [14]: [c_customer_sk#25, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_customer_sk#13] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(46) Project [codegen id : 5] Output [14]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38] Input [16]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#25, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38] @@ -288,20 +288,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [ca_address_sk#46, ca_state#47] Arguments: [ca_address_sk#46], [ca_address_sk#46] -(50) CometColumnarToRow [codegen id : 10] +(50) CometNativeColumnarToRow Input [1]: [ca_address_sk#46] (51) BroadcastExchange Input [1]: [ca_address_sk#46] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#27] Right keys [1]: [ca_address_sk#46] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(53) Project [codegen id : 5] Output [13]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38, ctr_total_return#15] Input [15]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38, ca_address_sk#46] @@ -313,7 +313,7 @@ Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIR Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (59) -+- * CometColumnarToRow (58) ++- CometNativeColumnarToRow (58) +- CometProject (57) +- CometFilter (56) +- CometNativeScan parquet spark_catalog.default.date_dim (55) @@ -334,7 +334,7 @@ Condition : ((isnotnull(d_year#48) AND (d_year#48 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#6, d_year#48] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(58) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (59) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt index 8373409822..c2e96eb412 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -19,28 +19,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -52,22 +52,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/simplified.txt index b7a7ece27e..b94ce87fd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (5) Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] @@ -8,10 +8,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] Project [wr_returning_customer_sk,wr_return_amt,ca_state] BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] @@ -23,37 +23,33 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre Scan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (8) + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] Project [wr_returning_customer_sk,wr_return_amt,ca_state] BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] @@ -70,17 +66,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometNativeColumnarToRow + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] InputAdapter BroadcastExchange #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/explain.txt index 44c6fdad28..5cb444e7c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/explain.txt @@ -285,7 +285,7 @@ Input [13]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt index 7ebd69d340..70bff8abb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/simplified.txt index da5ca97c85..cd44ec89e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 44c6fdad28..5cb444e7c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -285,7 +285,7 @@ Input [13]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/extended.txt index 7ebd69d340..70bff8abb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 da5ca97c85..cd44ec89e5 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/explain.txt index 9ce8c738e8..24cf234506 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (97) +CometNativeColumnarToRow (97) +- CometSort (96) +- CometColumnarExchange (95) +- * Project (94) @@ -11,7 +11,7 @@ : : : +- * BroadcastHashJoin Inner BuildRight (46) : : : :- * BroadcastHashJoin Inner BuildRight (31) : : : : :- * HashAggregate (16) - : : : : : +- * CometColumnarToRow (15) + : : : : : +- CometNativeColumnarToRow (15) : : : : : +- CometColumnarExchange (14) : : : : : +- * HashAggregate (13) : : : : : +- * Project (12) @@ -23,12 +23,12 @@ : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : : +- ReusedExchange (4) : : : : : +- BroadcastExchange (10) - : : : : : +- * CometColumnarToRow (9) + : : : : : +- CometNativeColumnarToRow (9) : : : : : +- CometFilter (8) : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : : : : +- BroadcastExchange (30) : : : : +- * HashAggregate (29) - : : : : +- * CometColumnarToRow (28) + : : : : +- CometNativeColumnarToRow (28) : : : : +- CometColumnarExchange (27) : : : : +- * HashAggregate (26) : : : : +- * Project (25) @@ -42,7 +42,7 @@ : : : : +- ReusedExchange (23) : : : +- BroadcastExchange (45) : : : +- * HashAggregate (44) - : : : +- * CometColumnarToRow (43) + : : : +- CometNativeColumnarToRow (43) : : : +- CometColumnarExchange (42) : : : +- * HashAggregate (41) : : : +- * Project (40) @@ -56,7 +56,7 @@ : : : +- ReusedExchange (38) : : +- BroadcastExchange (61) : : +- * HashAggregate (60) - : : +- * CometColumnarToRow (59) + : : +- CometNativeColumnarToRow (59) : : +- CometColumnarExchange (58) : : +- * HashAggregate (57) : : +- * Project (56) @@ -70,7 +70,7 @@ : : +- ReusedExchange (54) : +- BroadcastExchange (76) : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) + : +- CometNativeColumnarToRow (74) : +- CometColumnarExchange (73) : +- * HashAggregate (72) : +- * Project (71) @@ -84,7 +84,7 @@ : +- ReusedExchange (69) +- BroadcastExchange (92) +- * HashAggregate (91) - +- * CometColumnarToRow (90) + +- CometNativeColumnarToRow (90) +- CometColumnarExchange (89) +- * HashAggregate (88) +- * Project (87) @@ -106,23 +106,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_addr_sk#1) (4) ReusedExchange [Reuses operator id: 101] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] @@ -137,24 +137,24 @@ ReadSchema: struct Input [2]: [ca_address_sk#8, ca_county#9] Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) -(9) CometColumnarToRow [codegen id : 2] +(9) CometNativeColumnarToRow Input [2]: [ca_address_sk#8, ca_county#9] (10) BroadcastExchange Input [2]: [ca_address_sk#8, ca_county#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#1] Right keys [1]: [ca_address_sk#8] Join type: Inner Join condition: None -(12) Project [codegen id : 3] +(12) Project [codegen id : 1] Output [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 1] Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -165,10 +165,10 @@ Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(15) CometColumnarToRow [codegen id : 24] +(15) CometNativeColumnarToRow Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -(16) HashAggregate [codegen id : 24] +(16) HashAggregate [codegen id : 12] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -183,40 +183,40 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Condition : isnotnull(ss_addr_sk#14) (20) ReusedExchange [Reuses operator id: 105] Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#16] Right keys [1]: [d_date_sk#18] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] (23) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#21, ca_county#22] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_addr_sk#14] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] @@ -227,10 +227,10 @@ Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -(29) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 3] Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] Keys [3]: [ca_county#22, d_qoy#20, d_year#19] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] @@ -241,7 +241,7 @@ Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15) Input [2]: [ca_county#22, store_sales#25] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 24] +(31) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#22] Join type: Inner @@ -255,40 +255,40 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 10] +(33) ColumnarToRow [codegen id : 4] Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -(34) Filter [codegen id : 10] +(34) Filter [codegen id : 4] Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Condition : isnotnull(ss_addr_sk#26) (35) ReusedExchange [Reuses operator id: 109] Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#28] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 4] Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] (38) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#33, ca_county#34] -(39) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_addr_sk#26] Right keys [1]: [ca_address_sk#33] Join type: Inner Join condition: None -(40) Project [codegen id : 10] +(40) Project [codegen id : 4] Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] -(41) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 4] Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] @@ -299,10 +299,10 @@ Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometColumnarToRow [codegen id : 11] +(43) CometNativeColumnarToRow Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 5] Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] Keys [3]: [ca_county#34, d_qoy#32, d_year#31] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] @@ -313,13 +313,13 @@ Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27) Input [2]: [ca_county#34, store_sales#37] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 24] +(46) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#22] Right keys [1]: [ca_county#34] Join type: Inner Join condition: None -(47) Project [codegen id : 24] +(47) Project [codegen id : 12] Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] @@ -331,40 +331,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 14] +(49) ColumnarToRow [codegen id : 6] Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(50) Filter [codegen id : 14] +(50) Filter [codegen id : 6] Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Condition : isnotnull(ws_bill_addr_sk#38) (51) ReusedExchange [Reuses operator id: 101] Output [3]: [d_date_sk#41, d_year#42, d_qoy#43] -(52) BroadcastHashJoin [codegen id : 14] +(52) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#40] Right keys [1]: [d_date_sk#41] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 6] Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43] Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41, d_year#42, d_qoy#43] (54) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#44, ca_county#45] -(55) BroadcastHashJoin [codegen id : 14] +(55) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_bill_addr_sk#38] Right keys [1]: [ca_address_sk#44] Join type: Inner Join condition: None -(56) Project [codegen id : 14] +(56) Project [codegen id : 6] Output [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_address_sk#44, ca_county#45] -(57) HashAggregate [codegen id : 14] +(57) HashAggregate [codegen id : 6] Input [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] Keys [3]: [ca_county#45, d_qoy#43, d_year#42] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] @@ -375,10 +375,10 @@ Results [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] Arguments: hashpartitioning(ca_county#45, d_qoy#43, d_year#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(59) CometColumnarToRow [codegen id : 15] +(59) CometNativeColumnarToRow Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -(60) HashAggregate [codegen id : 15] +(60) HashAggregate [codegen id : 7] Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] Keys [3]: [ca_county#45, d_qoy#43, d_year#42] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] @@ -389,7 +389,7 @@ Results [2]: [ca_county#45, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39) Input [2]: [ca_county#45, web_sales#49] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(62) BroadcastHashJoin [codegen id : 24] +(62) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#45] Join type: Inner @@ -403,40 +403,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 18] +(64) ColumnarToRow [codegen id : 8] Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -(65) Filter [codegen id : 18] +(65) Filter [codegen id : 8] Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] Condition : isnotnull(ws_bill_addr_sk#50) (66) ReusedExchange [Reuses operator id: 105] Output [3]: [d_date_sk#53, d_year#54, d_qoy#55] -(67) BroadcastHashJoin [codegen id : 18] +(67) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_sold_date_sk#52] Right keys [1]: [d_date_sk#53] Join type: Inner Join condition: None -(68) Project [codegen id : 18] +(68) Project [codegen id : 8] Output [4]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55] Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54, d_qoy#55] (69) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#56, ca_county#57] -(70) BroadcastHashJoin [codegen id : 18] +(70) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ws_bill_addr_sk#50] Right keys [1]: [ca_address_sk#56] Join type: Inner Join condition: None -(71) Project [codegen id : 18] +(71) Project [codegen id : 8] Output [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_address_sk#56, ca_county#57] -(72) HashAggregate [codegen id : 18] +(72) HashAggregate [codegen id : 8] Input [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] Keys [3]: [ca_county#57, d_qoy#55, d_year#54] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#51))] @@ -447,10 +447,10 @@ Results [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] Arguments: hashpartitioning(ca_county#57, d_qoy#55, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(74) CometColumnarToRow [codegen id : 19] +(74) CometNativeColumnarToRow Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -(75) HashAggregate [codegen id : 19] +(75) HashAggregate [codegen id : 9] Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] Keys [3]: [ca_county#57, d_qoy#55, d_year#54] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#51))] @@ -461,13 +461,13 @@ Results [2]: [ca_county#57, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#51) Input [2]: [ca_county#57, web_sales#60] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] -(77) BroadcastHashJoin [codegen id : 24] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#45] Right keys [1]: [ca_county#57] Join type: Inner Join condition: (CASE WHEN (web_sales#49 > 0.00) THEN (web_sales#60 / web_sales#49) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) -(78) Project [codegen id : 24] +(78) Project [codegen id : 12] Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60] Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, ca_county#57, web_sales#60] @@ -479,40 +479,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 22] +(80) ColumnarToRow [codegen id : 10] Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -(81) Filter [codegen id : 22] +(81) Filter [codegen id : 10] Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] Condition : isnotnull(ws_bill_addr_sk#61) (82) ReusedExchange [Reuses operator id: 109] Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] -(83) BroadcastHashJoin [codegen id : 22] +(83) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#63] Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(84) Project [codegen id : 22] +(84) Project [codegen id : 10] Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65, d_qoy#66] (85) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#67, ca_county#68] -(86) BroadcastHashJoin [codegen id : 22] +(86) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_bill_addr_sk#61] Right keys [1]: [ca_address_sk#67] Join type: Inner Join condition: None -(87) Project [codegen id : 22] +(87) Project [codegen id : 10] Output [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_address_sk#67, ca_county#68] -(88) HashAggregate [codegen id : 22] +(88) HashAggregate [codegen id : 10] Input [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] Keys [3]: [ca_county#68, d_qoy#66, d_year#65] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] @@ -523,10 +523,10 @@ Results [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] Arguments: hashpartitioning(ca_county#68, d_qoy#66, d_year#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(90) CometColumnarToRow [codegen id : 23] +(90) CometNativeColumnarToRow Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -(91) HashAggregate [codegen id : 23] +(91) HashAggregate [codegen id : 11] Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] Keys [3]: [ca_county#68, d_qoy#66, d_year#65] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] @@ -537,13 +537,13 @@ Results [2]: [ca_county#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62) Input [2]: [ca_county#68, web_sales#71] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] -(93) BroadcastHashJoin [codegen id : 24] +(93) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#45] Right keys [1]: [ca_county#68] Join type: Inner Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#71 / web_sales#60) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) -(94) Project [codegen id : 24] +(94) Project [codegen id : 12] Output [6]: [ca_county#9, d_year#6, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#75] Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60, ca_county#68, web_sales#71] @@ -555,14 +555,14 @@ Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENT Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75], [ca_county#9 ASC NULLS FIRST] -(97) CometColumnarToRow [codegen id : 25] +(97) CometNativeColumnarToRow Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (101) -+- * CometColumnarToRow (100) ++- CometNativeColumnarToRow (100) +- CometFilter (99) +- CometNativeScan parquet spark_catalog.default.date_dim (98) @@ -578,7 +578,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(100) CometColumnarToRow [codegen id : 1] +(100) CometNativeColumnarToRow Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (101) BroadcastExchange @@ -587,7 +587,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 BroadcastExchange (105) -+- * CometColumnarToRow (104) ++- CometNativeColumnarToRow (104) +- CometFilter (103) +- CometNativeScan parquet spark_catalog.default.date_dim (102) @@ -603,7 +603,7 @@ ReadSchema: struct Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) -(104) CometColumnarToRow [codegen id : 1] +(104) CometNativeColumnarToRow Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] (105) BroadcastExchange @@ -612,7 +612,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (109) -+- * CometColumnarToRow (108) ++- CometNativeColumnarToRow (108) +- CometFilter (107) +- CometNativeScan parquet spark_catalog.default.date_dim (106) @@ -628,7 +628,7 @@ ReadSchema: struct Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) -(108) CometColumnarToRow [codegen id : 1] +(108) CometNativeColumnarToRow Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] (109) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt index f6f3c8032a..03bcf0413e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project @@ -10,7 +10,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- BroadcastHashJoin : : : : :- HashAggregate - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- HashAggregate : : : : : +- Project @@ -22,20 +22,20 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : : +- BroadcastExchange : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project @@ -47,20 +47,20 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- BroadcastExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -72,20 +72,20 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -97,16 +97,16 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -118,16 +118,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -139,11 +139,11 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/simplified.txt index f3a0572081..0a36656045 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/simplified.txt @@ -1,22 +1,75 @@ -WholeStageCodegen (25) - 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 - WholeStageCodegen (24) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] +CometNativeColumnarToRow + 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 + WholeStageCodegen (12) + Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] + Project [ca_county,d_year,store_sales,store_sales,store_sales] BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow + BroadcastHashJoin [ca_county,ca_county] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #2 + WholeStageCodegen (1) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [ca_address_sk,ca_county] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #6 + WholeStageCodegen (2) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #7 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #9 + WholeStageCodegen (4) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,d_year,d_qoy,ca_county] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -26,130 +79,45 @@ WholeStageCodegen (25) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #10 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #10 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (7) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #12 + WholeStageCodegen (6) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 + Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) + BroadcastExchange #13 + WholeStageCodegen (9) HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #14 + WholeStageCodegen (8) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,d_year,d_qoy,ca_county] BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] @@ -159,8 +127,30 @@ WholeStageCodegen (25) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 + ReusedExchange [d_date_sk,d_year,d_qoy] #7 InputAdapter ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (11) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ca_county,d_qoy,d_year] #16 + WholeStageCodegen (10) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #10 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/explain.txt index 83d717e295..d300161c8f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/explain.txt @@ -511,7 +511,7 @@ Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#6 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (94) -+- * CometColumnarToRow (93) ++- CometNativeColumnarToRow (93) +- CometFilter (92) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) CometColumnarToRow [codegen id : 1] +(93) CometNativeColumnarToRow Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometFilter (96) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * CometColumnarToRow (101) ++- CometNativeColumnarToRow (101) +- CometFilter (100) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(101) CometColumnarToRow [codegen id : 1] +(101) CometNativeColumnarToRow Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt index b4e7aa4db6..780504fb09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/simplified.txt index 2f97384320..745f9356ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/simplified.txt @@ -22,11 +22,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -45,11 +43,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -66,11 +62,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index 83d717e295..d300161c8f 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 @@ -511,7 +511,7 @@ Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#6 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (94) -+- * CometColumnarToRow (93) ++- CometNativeColumnarToRow (93) +- CometFilter (92) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) CometColumnarToRow [codegen id : 1] +(93) CometNativeColumnarToRow Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * CometColumnarToRow (97) ++- CometNativeColumnarToRow (97) +- CometFilter (96) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(97) CometColumnarToRow [codegen id : 1] +(97) CometNativeColumnarToRow Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * CometColumnarToRow (101) ++- CometNativeColumnarToRow (101) +- CometFilter (100) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(101) CometColumnarToRow [codegen id : 1] +(101) CometNativeColumnarToRow Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/extended.txt index b4e7aa4db6..780504fb09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -62,7 +62,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange 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 2f97384320..745f9356ce 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 @@ -22,11 +22,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -45,11 +43,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -66,11 +62,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 CometFilter [d_date_sk,d_year,d_qoy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/explain.txt index ea4c14a47b..771a2444c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * HashAggregate (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * Project (27) @@ -13,14 +13,14 @@ : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : +- BroadcastExchange (8) - : : +- * CometColumnarToRow (7) + : : +- CometNativeColumnarToRow (7) : : +- CometProject (6) : : +- CometFilter (5) : : +- CometNativeScan parquet spark_catalog.default.item (4) : +- BroadcastExchange (22) : +- * Filter (21) : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) + : +- CometNativeColumnarToRow (19) : +- CometColumnarExchange (18) : +- * HashAggregate (17) : +- * Project (16) @@ -40,10 +40,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 3] Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 3] Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) @@ -62,20 +62,20 @@ Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotn Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [i_item_sk#5] (8) BroadcastExchange Input [1]: [i_item_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 6] +(9) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 6] +(10) Project [codegen id : 3] Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] @@ -87,27 +87,27 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] +(12) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -(13) Filter [codegen id : 3] +(13) Filter [codegen id : 1] Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] Condition : isnotnull(cs_item_sk#7) (14) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#10] -(15) BroadcastHashJoin [codegen id : 3] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(16) Project [codegen id : 3] +(16) Project [codegen id : 1] Output [2]: [cs_item_sk#7, cs_ext_discount_amt#8] Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#10] -(17) HashAggregate [codegen id : 3] +(17) HashAggregate [codegen id : 1] Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] Keys [1]: [cs_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] @@ -118,17 +118,17 @@ Results [3]: [cs_item_sk#7, sum#13, count#14] Input [3]: [cs_item_sk#7, sum#13, count#14] Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) CometColumnarToRow [codegen id : 4] +(19) CometNativeColumnarToRow Input [3]: [cs_item_sk#7, sum#13, count#14] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [cs_item_sk#7, sum#13, count#14] Keys [1]: [cs_item_sk#7] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#15] Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -(21) Filter [codegen id : 4] +(21) Filter [codegen id : 2] Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) @@ -136,30 +136,30 @@ Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#5] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16) -(24) Project [codegen id : 6] +(24) Project [codegen id : 3] Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] (25) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#10] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 3] Output [1]: [cs_ext_discount_amt#2] Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#10] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 3] Input [1]: [cs_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] @@ -170,10 +170,10 @@ Results [1]: [sum#18] Input [1]: [sum#18] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [1]: [sum#18] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 4] Input [1]: [sum#18] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] @@ -184,7 +184,7 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#19,17,2) AS Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -205,7 +205,7 @@ Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 2000-01-27)) AND (d_date#2 Input [2]: [d_date_sk#10, d_date#21] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt index 1eb8f397c3..f7096f5c3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -13,19 +13,19 @@ HashAggregate : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -35,12 +35,12 @@ HashAggregate : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/simplified.txt index 14c21b32a5..c9760ec371 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [cs_ext_discount_amt] [sum,sum] Project [cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -17,29 +17,25 @@ WholeStageCodegen (7) Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(cs_ext_discount_amt))] HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cs_item_sk] #5 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] Project [cs_item_sk,cs_ext_discount_amt] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/explain.txt index b27d40f91b..44ee043734 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/explain.txt @@ -175,7 +175,7 @@ Input [1]: [excess discount amount#17] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt index 943bb147ff..c7273327fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/simplified.txt index 07619fc999..78be4baea7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/simplified.txt @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index b27d40f91b..44ee043734 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -175,7 +175,7 @@ Input [1]: [excess discount amount#17] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/extended.txt index 943bb147ff..c7273327fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 07619fc999..78be4baea7 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 @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/explain.txt index f441da7c27..dae715c025 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (74) +- * HashAggregate (73) - +- * CometColumnarToRow (72) + +- CometNativeColumnarToRow (72) +- CometColumnarExchange (71) +- * HashAggregate (70) +- Union (69) :- * HashAggregate (28) - : +- * CometColumnarToRow (27) + : +- CometNativeColumnarToRow (27) : +- CometColumnarExchange (26) : +- * HashAggregate (25) : +- * Project (24) @@ -20,12 +20,12 @@ TakeOrderedAndProject (74) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometBroadcastHashJoin (20) : :- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.item (14) @@ -34,7 +34,7 @@ TakeOrderedAndProject (74) : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.item (16) :- * HashAggregate (52) - : +- * CometColumnarToRow (51) + : +- CometNativeColumnarToRow (51) : +- CometColumnarExchange (50) : +- * HashAggregate (49) : +- * Project (48) @@ -49,7 +49,7 @@ TakeOrderedAndProject (74) : : : +- ReusedExchange (32) : : +- ReusedExchange (35) : +- BroadcastExchange (46) - : +- * CometColumnarToRow (45) + : +- CometNativeColumnarToRow (45) : +- CometBroadcastHashJoin (44) : :- CometFilter (39) : : +- CometNativeScan parquet spark_catalog.default.item (38) @@ -58,7 +58,7 @@ TakeOrderedAndProject (74) : +- CometFilter (41) : +- CometNativeScan parquet spark_catalog.default.item (40) +- * HashAggregate (68) - +- * CometColumnarToRow (67) + +- CometNativeColumnarToRow (67) +- CometColumnarExchange (66) +- * HashAggregate (65) +- * Project (64) @@ -83,23 +83,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] @@ -118,20 +118,20 @@ Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isno Input [2]: [ca_address_sk#7, ca_gmt_offset#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [ca_address_sk#7] (11) BroadcastExchange Input [1]: [ca_address_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] @@ -169,24 +169,24 @@ Left output [2]: [i_item_sk#9, i_manufact_id#10] Right output [1]: [i_manufact_id#10#12] Arguments: [i_manufact_id#10], [i_manufact_id#10#12], LeftSemi, BuildRight -(21) CometColumnarToRow [codegen id : 3] +(21) CometNativeColumnarToRow Input [2]: [i_item_sk#9, i_manufact_id#10] (22) BroadcastExchange Input [2]: [i_item_sk#9, i_manufact_id#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 4] +(23) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#9] Join type: Inner Join condition: None -(24) Project [codegen id : 4] +(24) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#3, i_manufact_id#10] Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_manufact_id#10] -(25) HashAggregate [codegen id : 4] +(25) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_manufact_id#10] Keys [1]: [i_manufact_id#10] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -197,10 +197,10 @@ Results [2]: [i_manufact_id#10, sum#14] Input [2]: [i_manufact_id#10, sum#14] Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(27) CometColumnarToRow [codegen id : 5] +(27) CometNativeColumnarToRow Input [2]: [i_manufact_id#10, sum#14] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 2] Input [2]: [i_manufact_id#10, sum#14] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -215,36 +215,36 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 9] +(30) ColumnarToRow [codegen id : 3] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -(31) Filter [codegen id : 9] +(31) Filter [codegen id : 3] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) (32) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#21] -(33) BroadcastHashJoin [codegen id : 9] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#20] Right keys [1]: [d_date_sk#21] Join type: Inner Join condition: None -(34) Project [codegen id : 9] +(34) Project [codegen id : 3] Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#21] (35) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#22] -(36) BroadcastHashJoin [codegen id : 9] +(36) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_addr_sk#17] Right keys [1]: [ca_address_sk#22] Join type: Inner Join condition: None -(37) Project [codegen id : 9] +(37) Project [codegen id : 3] Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#22] @@ -282,24 +282,24 @@ Left output [2]: [i_item_sk#23, i_manufact_id#24] Right output [1]: [i_manufact_id#10] Arguments: [i_manufact_id#24], [i_manufact_id#10], LeftSemi, BuildRight -(45) CometColumnarToRow [codegen id : 8] +(45) CometNativeColumnarToRow Input [2]: [i_item_sk#23, i_manufact_id#24] (46) BroadcastExchange Input [2]: [i_item_sk#23, i_manufact_id#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(47) BroadcastHashJoin [codegen id : 9] +(47) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#18] Right keys [1]: [i_item_sk#23] Join type: Inner Join condition: None -(48) Project [codegen id : 9] +(48) Project [codegen id : 3] Output [2]: [cs_ext_sales_price#19, i_manufact_id#24] Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#23, i_manufact_id#24] -(49) HashAggregate [codegen id : 9] +(49) HashAggregate [codegen id : 3] Input [2]: [cs_ext_sales_price#19, i_manufact_id#24] Keys [1]: [i_manufact_id#24] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] @@ -310,10 +310,10 @@ Results [2]: [i_manufact_id#24, sum#26] Input [2]: [i_manufact_id#24, sum#26] Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(51) CometColumnarToRow [codegen id : 10] +(51) CometNativeColumnarToRow Input [2]: [i_manufact_id#24, sum#26] -(52) HashAggregate [codegen id : 10] +(52) HashAggregate [codegen id : 4] Input [2]: [i_manufact_id#24, sum#26] Keys [1]: [i_manufact_id#24] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] @@ -328,53 +328,53 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 14] +(54) ColumnarToRow [codegen id : 5] Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] -(55) Filter [codegen id : 14] +(55) Filter [codegen id : 5] Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) (56) ReusedExchange [Reuses operator id: 79] Output [1]: [d_date_sk#33] -(57) BroadcastHashJoin [codegen id : 14] +(57) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#32] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(58) Project [codegen id : 14] +(58) Project [codegen id : 5] Output [3]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31] Input [5]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32, d_date_sk#33] (59) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#34] -(60) BroadcastHashJoin [codegen id : 14] +(60) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_bill_addr_sk#30] Right keys [1]: [ca_address_sk#34] Join type: Inner Join condition: None -(61) Project [codegen id : 14] +(61) Project [codegen id : 5] Output [2]: [ws_item_sk#29, ws_ext_sales_price#31] Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ca_address_sk#34] (62) ReusedExchange [Reuses operator id: 46] Output [2]: [i_item_sk#35, i_manufact_id#36] -(63) BroadcastHashJoin [codegen id : 14] +(63) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_item_sk#29] Right keys [1]: [i_item_sk#35] Join type: Inner Join condition: None -(64) Project [codegen id : 14] +(64) Project [codegen id : 5] Output [2]: [ws_ext_sales_price#31, i_manufact_id#36] Input [4]: [ws_item_sk#29, ws_ext_sales_price#31, i_item_sk#35, i_manufact_id#36] -(65) HashAggregate [codegen id : 14] +(65) HashAggregate [codegen id : 5] Input [2]: [ws_ext_sales_price#31, i_manufact_id#36] Keys [1]: [i_manufact_id#36] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#31))] @@ -385,10 +385,10 @@ Results [2]: [i_manufact_id#36, sum#38] Input [2]: [i_manufact_id#36, sum#38] Arguments: hashpartitioning(i_manufact_id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(67) CometColumnarToRow [codegen id : 15] +(67) CometNativeColumnarToRow Input [2]: [i_manufact_id#36, sum#38] -(68) HashAggregate [codegen id : 15] +(68) HashAggregate [codegen id : 6] Input [2]: [i_manufact_id#36, sum#38] Keys [1]: [i_manufact_id#36] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#31))] @@ -397,7 +397,7 @@ Results [2]: [i_manufact_id#36, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price (69) Union -(70) HashAggregate [codegen id : 16] +(70) HashAggregate [codegen id : 7] Input [2]: [i_manufact_id#10, total_sales#16] Keys [1]: [i_manufact_id#10] Functions [1]: [partial_sum(total_sales#16)] @@ -408,10 +408,10 @@ Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(72) CometColumnarToRow [codegen id : 17] +(72) CometNativeColumnarToRow Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -(73) HashAggregate [codegen id : 17] +(73) HashAggregate [codegen id : 8] Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] Keys [1]: [i_manufact_id#10] Functions [1]: [sum(total_sales#16)] @@ -426,7 +426,7 @@ Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_manufact_id#10, total_sales Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometFilter (76) +- CometNativeScan parquet spark_catalog.default.date_dim (75) @@ -447,7 +447,7 @@ Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1 Input [3]: [d_date_sk#6, d_year#47, d_moy#48] Arguments: [d_date_sk#6], [d_date_sk#6] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt index 79b2700a8c..38b3efb123 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,22 +19,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -43,7 +43,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -57,17 +57,17 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -76,7 +76,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -90,17 +90,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/simplified.txt index 719ccdbb9f..61459d30c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_manufact_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -27,40 +27,34 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #6 - CometProject [i_manufact_id] [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (10) + CometNativeColumnarToRow + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [i_manufact_id] #6 + CometProject [i_manufact_id] [i_manufact_id] + CometFilter [i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (4) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #7 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -79,22 +73,20 @@ TakeOrderedAndProject [total_sales,i_manufact_id] ReusedExchange [ca_address_sk] #4 InputAdapter BroadcastExchange #8 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #9 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (15) + CometNativeColumnarToRow + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [i_manufact_id] #9 + CometProject [i_manufact_id] + CometFilter [i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (6) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_manufact_id] #10 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] BroadcastHashJoin [ws_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/explain.txt index b78621c4bb..edf25d0127 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/explain.txt @@ -387,7 +387,7 @@ Input [2]: [i_manufact_id#12, total_sales#41] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (73) -+- * CometColumnarToRow (72) ++- CometNativeColumnarToRow (72) +- CometProject (71) +- CometFilter (70) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) @@ -408,7 +408,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(72) CometColumnarToRow [codegen id : 1] +(72) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (73) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt index 063fdb6faf..971bf3c1f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/simplified.txt index ff7856615f..a9f11a05d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index b78621c4bb..edf25d0127 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -387,7 +387,7 @@ Input [2]: [i_manufact_id#12, total_sales#41] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (73) -+- * CometColumnarToRow (72) ++- CometNativeColumnarToRow (72) +- CometProject (71) +- CometFilter (70) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) @@ -408,7 +408,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(72) CometColumnarToRow [codegen id : 1] +(72) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (73) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/extended.txt index 063fdb6faf..971bf3c1f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ff7856615f..a9f11a05d6 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/explain.txt index 18c904ceed..7f5f099e1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometSort (34) +- CometColumnarExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) :- * Filter (25) : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -20,17 +20,17 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.customer (26) @@ -44,23 +44,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] @@ -79,20 +79,20 @@ Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND is Input [2]: [s_store_sk#8, s_county#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (11) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] @@ -111,24 +111,24 @@ Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apac Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#10] (18) BroadcastExchange Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] @@ -139,17 +139,17 @@ Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) @@ -168,20 +168,20 @@ Condition : isnotnull(c_customer_sk#18) Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] (30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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] @@ -193,14 +193,14 @@ Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC 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: [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 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (40) -+- * CometColumnarToRow (39) ++- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometNativeScan parquet spark_catalog.default.date_dim (36) @@ -221,7 +221,7 @@ Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(39) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (40) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt index 9e4596c12c..40230582bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- BroadcastHashJoin :- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,27 +19,27 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/simplified.txt index 10af5ceadc..bbcdf458c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/simplified.txt @@ -1,59 +1,49 @@ -WholeStageCodegen (7) - 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 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] +CometNativeColumnarToRow + 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 + WholeStageCodegen (2) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/explain.txt index 287e323952..b1fe282f8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/simplified.txt index 47d0e35dd2..3362a73ddd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 287e323952..b1fe282f8b 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 @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 47d0e35dd2..3362a73ddd 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/explain.txt index c65bb26de3..1980057652 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -13,7 +13,7 @@ TakeOrderedAndProject (45) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -35,12 +35,12 @@ TakeOrderedAndProject (45) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_address (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) @@ -57,7 +57,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] (6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] @@ -87,7 +87,7 @@ Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi @@ -100,19 +100,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] (13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#9] @@ -120,7 +120,7 @@ Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#9] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) @@ -133,19 +133,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] (20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#12] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#9] @@ -153,17 +153,17 @@ Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#9] Input [1]: [cs_ship_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#12] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(25) Filter [codegen id : 4] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -182,20 +182,20 @@ Condition : isnotnull(ca_address_sk#14) Input [2]: [ca_address_sk#14, ca_state#15] Arguments: [ca_address_sk#14, ca_state#16], [ca_address_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#15, 2, true, false, true) AS ca_state#16] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [2]: [ca_address_sk#14, ca_state#16] (31) BroadcastExchange Input [2]: [ca_address_sk#14, ca_state#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#14] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, ca_state#16] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#14, ca_state#16] @@ -214,24 +214,24 @@ Condition : isnotnull(cd_demo_sk#17) Input [6]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Arguments: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22], [cd_demo_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#18, 1, true, false, true) AS cd_gender#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [6]: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] (38) BroadcastExchange Input [6]: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#17] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 4] Output [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Input [8]: [c_current_cdemo_sk#4, ca_state#16, cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 4] Input [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Keys [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Functions [10]: [partial_count(1), partial_min(cd_dep_count#20), partial_max(cd_dep_count#20), partial_avg(cd_dep_count#20), partial_min(cd_dep_employed_count#21), partial_max(cd_dep_employed_count#21), partial_avg(cd_dep_employed_count#21), partial_min(cd_dep_college_count#22), partial_max(cd_dep_college_count#22), partial_avg(cd_dep_college_count#22)] @@ -242,10 +242,10 @@ Results [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, min#39, max#40, sum#41, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50] Arguments: hashpartitioning(ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, min#39, max#40, sum#41, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 5] Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, min#39, max#40, sum#41, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50] Keys [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Functions [10]: [count(1), min(cd_dep_count#20), max(cd_dep_count#20), avg(cd_dep_count#20), min(cd_dep_employed_count#21), max(cd_dep_employed_count#21), avg(cd_dep_employed_count#21), min(cd_dep_college_count#22), max(cd_dep_college_count#22), avg(cd_dep_college_count#22)] @@ -260,7 +260,7 @@ Arguments: 100, [ca_state#16 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * CometColumnarToRow (49) ++- CometNativeColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometNativeScan parquet spark_catalog.default.date_dim (46) @@ -281,7 +281,7 @@ Condition : ((((isnotnull(d_year#73) AND isnotnull(d_qoy#74)) AND (d_year#73 = 2 Input [3]: [d_date_sk#9, d_year#73, d_qoy#74] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt index 8935b6cc13..40a8ef4a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -22,12 +22,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,17 +49,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/simplified.txt index 48bc5637d0..a0b0348e6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -29,17 +29,15 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,7 +48,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -61,17 +59,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/explain.txt index f01bb79a29..92cab577a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#15] Input [1]: [cs_ship_customer_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#15] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#18) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#18, ca_state#20] (33) BroadcastExchange Input [2]: [ca_address_sk#18, ca_state#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#20] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#21) Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] (40) BroadcastExchange Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#21] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [10]: [partial_count(1), partial_min(cd_dep_count#24), partial_max(cd_dep_count#24), partial_avg(cd_dep_count#24), partial_min(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_avg(cd_dep_employed_count#25), partial_min(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_avg(cd_dep_college_count#26)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [10]: [count(1), min(cd_dep_count#24), max(cd_dep_count#24), avg(cd_dep_count#24), min(cd_dep_employed_count#25), max(cd_dep_employed_count#25), avg(cd_dep_employed_count#25), min(cd_dep_college_count#26), max(cd_dep_college_count#26), avg(cd_dep_college_count#26)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/simplified.txt index f1fe09fb46..4acbd21997 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index f01bb79a29..92cab577a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#15] Input [1]: [cs_ship_customer_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#15] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#18) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#18, ca_state#20] (33) BroadcastExchange Input [2]: [ca_address_sk#18, ca_state#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#20] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#21) Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] (40) BroadcastExchange Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#21] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [10]: [partial_count(1), partial_min(cd_dep_count#24), partial_max(cd_dep_count#24), partial_avg(cd_dep_count#24), partial_min(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_avg(cd_dep_employed_count#25), partial_min(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_avg(cd_dep_college_count#26)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [10]: [count(1), min(cd_dep_count#24), max(cd_dep_count#24), avg(cd_dep_count#24), min(cd_dep_employed_count#25), max(cd_dep_employed_count#25), avg(cd_dep_employed_count#25), min(cd_dep_college_count#26), max(cd_dep_college_count#26), avg(cd_dep_college_count#26)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt index f1fe09fb46..4acbd21997 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/explain.txt index 58c6701f56..5c2b8c1ef6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (31) +- * Project (30) +- Window (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometSort (27) +- CometColumnarExchange (26) +- * HashAggregate (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometColumnarExchange (23) +- * HashAggregate (22) +- * Expand (21) @@ -21,12 +21,12 @@ TakeOrderedAndProject (31) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometProject (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -40,23 +40,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (4) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] @@ -75,20 +75,20 @@ Condition : isnotnull(i_item_sk#8) Input [3]: [i_item_sk#8, i_class#9, i_category#10] Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#8, i_class#11, i_category#12] (11) BroadcastExchange Input [3]: [i_item_sk#8, i_class#11, i_category#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] @@ -107,28 +107,28 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#13, s_state#14] Arguments: [s_store_sk#13], [s_store_sk#13] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [s_store_sk#13] (18) BroadcastExchange Input [1]: [s_store_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] -(21) Expand [codegen id : 4] +(21) Expand [codegen id : 1] Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 1] Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -139,10 +139,10 @@ Results [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] Arguments: hashpartitioning(i_category#15, i_class#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 5] +(24) CometNativeColumnarToRow Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 2] Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] @@ -157,14 +157,14 @@ Arguments: hashpartitioning(_w1#27, _w2#28, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] Arguments: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28], [_w1#27 ASC NULLS FIRST, _w2#28 ASC NULLS FIRST, _w0#26 ASC NULLS FIRST] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] (29) Window Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] Arguments: [rank(_w0#26) windowspecdefinition(_w1#27, _w2#28, _w0#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#29], [_w1#27, _w2#28], [_w0#26 ASC NULLS FIRST] -(30) Project [codegen id : 7] +(30) Project [codegen id : 3] Output [5]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] Input [8]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28, rank_within_parent#29] @@ -176,7 +176,7 @@ Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#30) AND (d_year#30 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#7, d_year#30] Arguments: [d_date_sk#7], [d_date_sk#7] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt index bd826eb3a3..aa40ed68cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -20,22 +20,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/simplified.txt index d6f8548c89..a6427d7111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/simplified.txt @@ -1,54 +1,46 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) + WholeStageCodegen (3) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] + Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt index ae1d4c242b..401c5b564e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt @@ -172,7 +172,7 @@ Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -193,7 +193,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt index 89c523a388..2bf9a90726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt index 61782e3011..248d0bc6e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index ae1d4c242b..401c5b564e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt @@ -172,7 +172,7 @@ Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -193,7 +193,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt index 89c523a388..2bf9a90726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 61782e3011..248d0bc6e8 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 @@ -22,12 +22,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/explain.txt index e36c995169..17981c8b7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (26) +CometNativeColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometColumnarExchange (23) @@ -11,7 +11,7 @@ : +- * BroadcastHashJoin Inner BuildRight (13) : :- * Project (11) : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -21,7 +21,7 @@ : : +- * ColumnarToRow (6) : : +- Scan parquet spark_catalog.default.inventory (5) : +- ReusedExchange (12) - +- * CometColumnarToRow (19) + +- CometNativeColumnarToRow (19) +- CometProject (18) +- CometFilter (17) +- CometNativeScan parquet spark_catalog.default.catalog_sales (16) @@ -42,7 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] (5) Scan parquet spark_catalog.default.inventory @@ -68,26 +68,26 @@ Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] Input [2]: [inv_item_sk#7, inv_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [inv_item_sk#7] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] (12) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#11] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] @@ -110,20 +110,20 @@ Condition : isnotnull(cs_item_sk#12) Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] Arguments: [cs_item_sk#12], [cs_item_sk#12] -(19) CometColumnarToRow +(19) CometNativeColumnarToRow Input [1]: [cs_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#12] Join type: Inner Join condition: None -(21) Project [codegen id : 4] +(21) Project [codegen id : 3] Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#12] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 3] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Functions: [] @@ -143,14 +143,14 @@ Functions: [] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] -(26) CometColumnarToRow [codegen id : 5] +(26) CometNativeColumnarToRow Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.date_dim (27) @@ -171,7 +171,7 @@ Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#14] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt index fb7dbb8d72..296b2c8117 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -10,7 +10,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -21,16 +21,16 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_sales diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/simplified.txt index 88a3895990..3f16bc9b50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/simplified.txt @@ -1,45 +1,41 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,cs_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [inv_item_sk,inv_date_sk] + Filter [inv_quantity_on_hand,inv_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + CometNativeColumnarToRow + CometProject [cs_item_sk] + CometFilter [cs_item_sk,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/explain.txt index e4cfdbf6d1..c550eec11c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt index abe0937d1d..be3d6196b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/simplified.txt index 7ab646a628..f5ddbb2d04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt index e4cfdbf6d1..c550eec11c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/extended.txt index abe0937d1d..be3d6196b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt index 7ab646a628..f5ddbb2d04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/explain.txt index f1a8620998..4d03a1f573 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (49) +CometNativeColumnarToRow (49) +- CometHashAggregate (48) +- CometExchange (47) +- CometHashAggregate (46) @@ -18,7 +18,7 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer (7) @@ -58,23 +58,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) (4) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#4, d_date#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#2] Right keys [1]: [d_date_sk#4] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, d_date#5] Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] @@ -93,24 +93,24 @@ Condition : isnotnull(c_customer_sk#6) Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#7, 20, true, false, true) AS c_first_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#8, 30, true, false, true) AS c_last_name#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] (11) BroadcastExchange Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [c_last_name#10, c_first_name#9, d_date#5] Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [c_last_name#10, c_first_name#9, d_date#5] Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] Functions: [] @@ -134,40 +134,40 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Condition : isnotnull(cs_bill_customer_sk#11) (20) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#13, d_date#14] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [2]: [cs_bill_customer_sk#11, d_date#14] Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] (23) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_bill_customer_sk#11] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [3]: [c_last_name#17, c_first_name#16, d_date#14] Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] @@ -200,40 +200,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 9] +(32) ColumnarToRow [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(33) Filter [codegen id : 9] +(33) Filter [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Condition : isnotnull(ws_bill_customer_sk#18) (34) ReusedExchange [Reuses operator id: 54] Output [2]: [d_date_sk#20, d_date#21] -(35) BroadcastHashJoin [codegen id : 9] +(35) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(36) Project [codegen id : 9] +(36) Project [codegen id : 3] Output [2]: [ws_bill_customer_sk#18, d_date#21] Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] (37) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(38) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_bill_customer_sk#18] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(39) Project [codegen id : 3] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 3] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] @@ -275,14 +275,14 @@ Input [1]: [count#25] Keys: [] Functions [1]: [count(1)] -(49) CometColumnarToRow [codegen id : 10] +(49) CometNativeColumnarToRow Input [1]: [count(1)#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (54) -+- * CometColumnarToRow (53) ++- CometNativeColumnarToRow (53) +- CometProject (52) +- CometFilter (51) +- CometNativeScan parquet spark_catalog.default.date_dim (50) @@ -303,7 +303,7 @@ Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_mo Input [3]: [d_date_sk#4, d_date#5, d_month_seq#27] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(53) CometColumnarToRow [codegen id : 1] +(53) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (54) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt index 061475900e..6d9f5c7a17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -17,17 +17,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -44,12 +44,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer @@ -66,12 +66,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/simplified.txt index 547965d09c..f127e42be4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/simplified.txt @@ -1,75 +1,69 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] +CometNativeColumnarToRow + CometHashAggregate [count] [count(1),count(1)] + CometExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (1) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #5 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange [c_last_name,c_first_name,d_date] #5 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + CometBroadcastExchange [c_last_name,c_first_name,d_date] #7 + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/explain.txt index 591fa114fb..1ca0f51516 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/explain.txt @@ -273,7 +273,7 @@ Input [1]: [count(1)#29] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -294,7 +294,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt index 55af64efc6..715ff39db7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/simplified.txt index 7ceccf5d2c..6c90e8467d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index 591fa114fb..1ca0f51516 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -273,7 +273,7 @@ Input [1]: [count(1)#29] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -294,7 +294,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/extended.txt index 55af64efc6..715ff39db7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 7ceccf5d2c..6c90e8467d 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/explain.txt index b8ed3a361b..89eac3d545 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- * BroadcastHashJoin Inner BuildRight (44) :- * Project (24) : +- * Filter (23) : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometColumnarExchange (20) : +- * HashAggregate (19) : +- * Project (18) @@ -19,11 +19,11 @@ : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.inventory (1) : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) + : : : +- CometNativeColumnarToRow (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) + : : +- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : +- ReusedExchange (16) @@ -31,7 +31,7 @@ +- * Project (42) +- * Filter (41) +- * HashAggregate (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometColumnarExchange (38) +- * HashAggregate (37) +- * Project (36) @@ -56,10 +56,10 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) @@ -74,20 +74,20 @@ ReadSchema: struct Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [i_item_sk#6] (7) BroadcastExchange Input [1]: [i_item_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] @@ -102,37 +102,37 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] (13) BroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] (16) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#9, d_moy#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] @@ -143,21 +143,21 @@ Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometColumnarToRow [codegen id : 10] +(21) CometNativeColumnarToRow Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(22) HashAggregate [codegen id : 10] +(22) HashAggregate [codegen id : 4] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(23) Filter [codegen id : 10] +(23) Filter [codegen id : 4] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END -(24) Project [codegen id : 10] +(24) Project [codegen id : 4] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -169,53 +169,53 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 8] +(26) ColumnarToRow [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(27) Filter [codegen id : 8] +(27) Filter [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) (28) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(29) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(30) Project [codegen id : 8] +(30) Project [codegen id : 2] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] (31) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] (34) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#34, d_moy#35] -(35) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(36) Project [codegen id : 8] +(36) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(37) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 2] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] @@ -226,21 +226,21 @@ Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n# Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) CometColumnarToRow [codegen id : 9] +(39) CometNativeColumnarToRow Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 3] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] -(41) Filter [codegen id : 9] +(41) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END -(42) Project [codegen id : 9] +(42) Project [codegen id : 3] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] @@ -248,7 +248,7 @@ Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(44) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner @@ -262,14 +262,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 11] +(47) CometNativeColumnarToRow Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometNativeScan parquet spark_catalog.default.date_dim (48) @@ -290,7 +290,7 @@ Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2 Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#10] (52) BroadcastExchange @@ -299,7 +299,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -320,7 +320,7 @@ Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2 Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#34, d_moy#35] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt index e44cca98a1..d9cec17b28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- BroadcastHashJoin :- Project : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,20 +19,20 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,7 +40,7 @@ CometColumnarToRow +- Project +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -54,20 +54,20 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/simplified.txt index 11dd59a97f..ff7f9077d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/simplified.txt @@ -1,84 +1,74 @@ -WholeStageCodegen (11) - 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 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + 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 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (1) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (2) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk] #4 + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + ReusedExchange [i_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/explain.txt index ed74780c2c..fce6b102d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/explain.txt @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/simplified.txt index 2cd1f70c8a..0c4aa97a2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 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 ed74780c2c..fce6b102d0 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 @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 2cd1f70c8a..0c4aa97a2b 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/explain.txt index 69fbdb52f0..4510aa3b77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- * BroadcastHashJoin Inner BuildRight (44) :- * Project (24) : +- * Filter (23) : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometColumnarExchange (20) : +- * HashAggregate (19) : +- * Project (18) @@ -19,11 +19,11 @@ : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.inventory (1) : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) + : : : +- CometNativeColumnarToRow (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) + : : +- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : +- ReusedExchange (16) @@ -31,7 +31,7 @@ +- * Project (42) +- * Filter (41) +- * HashAggregate (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometColumnarExchange (38) +- * HashAggregate (37) +- * Project (36) @@ -56,10 +56,10 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) @@ -74,20 +74,20 @@ ReadSchema: struct Input [1]: [i_item_sk#6] Condition : isnotnull(i_item_sk#6) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [i_item_sk#6] (7) BroadcastExchange Input [1]: [i_item_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 4] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(9) Project [codegen id : 4] +(9) Project [codegen id : 1] Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] @@ -102,37 +102,37 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Condition : isnotnull(w_warehouse_sk#7) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] (13) BroadcastExchange Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] (16) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#9, d_moy#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] -(19) HashAggregate [codegen id : 4] +(19) HashAggregate [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] @@ -143,21 +143,21 @@ Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) CometColumnarToRow [codegen id : 10] +(21) CometNativeColumnarToRow Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -(22) HashAggregate [codegen id : 10] +(22) HashAggregate [codegen id : 4] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] -(23) Filter [codegen id : 10] +(23) Filter [codegen id : 4] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.5) END) -(24) Project [codegen id : 10] +(24) Project [codegen id : 4] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] @@ -169,53 +169,53 @@ PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 8] +(26) ColumnarToRow [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -(27) Filter [codegen id : 8] +(27) Filter [codegen id : 2] Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) (28) ReusedExchange [Reuses operator id: 7] Output [1]: [i_item_sk#31] -(29) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_item_sk#26] Right keys [1]: [i_item_sk#31] Join type: Inner Join condition: None -(30) Project [codegen id : 8] +(30) Project [codegen id : 2] Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] (31) ReusedExchange [Reuses operator id: 13] Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] -(32) BroadcastHashJoin [codegen id : 8] +(32) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_warehouse_sk#27] Right keys [1]: [w_warehouse_sk#32] Join type: Inner Join condition: None -(33) Project [codegen id : 8] +(33) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] (34) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#34, d_moy#35] -(35) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(36) Project [codegen id : 8] +(36) Project [codegen id : 2] Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] -(37) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 2] Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] @@ -226,21 +226,21 @@ Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n# Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) CometColumnarToRow [codegen id : 9] +(39) CometNativeColumnarToRow Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 3] Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] -(41) Filter [codegen id : 9] +(41) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END -(42) Project [codegen id : 9] +(42) Project [codegen id : 3] Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#47] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] @@ -248,7 +248,7 @@ Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] -(44) BroadcastHashJoin [codegen id : 10] +(44) BroadcastHashJoin [codegen id : 4] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] Join type: Inner @@ -262,14 +262,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 11] +(47) CometNativeColumnarToRow Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometNativeScan parquet spark_catalog.default.date_dim (48) @@ -290,7 +290,7 @@ Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2 Input [3]: [d_date_sk#9, d_year#48, d_moy#10] Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#10] (52) BroadcastExchange @@ -299,7 +299,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -320,7 +320,7 @@ Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2 Input [3]: [d_date_sk#34, d_year#49, d_moy#35] Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#34, d_moy#35] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt index e44cca98a1..d9cec17b28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- BroadcastHashJoin :- Project : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,20 +19,20 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,7 +40,7 @@ CometColumnarToRow +- Project +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -54,20 +54,20 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/simplified.txt index 11dd59a97f..ff7f9077d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/simplified.txt @@ -1,84 +1,74 @@ -WholeStageCodegen (11) - 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 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + 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 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (1) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (2) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Filter [inv_item_sk,inv_warehouse_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk] #4 + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + ReusedExchange [i_item_sk] #4 InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/explain.txt index 2a31b0d8b2..b2934e5cae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/explain.txt @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/simplified.txt index 2cd1f70c8a..0c4aa97a2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 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 2a31b0d8b2..b2934e5cae 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 @@ -268,7 +268,7 @@ Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehou Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_moy#11] Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 Input [3]: [d_date_sk#28, d_year#29, d_moy#30] Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/extended.txt index 8ff1442cf8..8a5a4af707 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 2cd1f70c8a..0c4aa97a2b 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [i_item_sk] #4 CometFilter [i_item_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] @@ -51,12 +49,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] ReusedExchange [i_item_sk] #4 ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 CometBroadcastExchange [d_date_sk,d_moy] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/explain.txt index 9fffd1dde5..926367ac8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/explain.txt @@ -11,14 +11,14 @@ TakeOrderedAndProject (120) : : : :- * BroadcastHashJoin Inner BuildRight (37) : : : : :- * Filter (18) : : : : : +- * HashAggregate (17) - : : : : : +- * CometColumnarToRow (16) + : : : : : +- CometNativeColumnarToRow (16) : : : : : +- CometColumnarExchange (15) : : : : : +- * HashAggregate (14) : : : : : +- * Project (13) : : : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : : : :- * Project (10) : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * CometColumnarToRow (4) + : : : : : : :- CometNativeColumnarToRow (4) : : : : : : : +- CometProject (3) : : : : : : : +- CometFilter (2) : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -29,14 +29,14 @@ TakeOrderedAndProject (120) : : : : : +- ReusedExchange (11) : : : : +- BroadcastExchange (36) : : : : +- * HashAggregate (35) - : : : : +- * CometColumnarToRow (34) + : : : : +- CometNativeColumnarToRow (34) : : : : +- CometColumnarExchange (33) : : : : +- * HashAggregate (32) : : : : +- * Project (31) : : : : +- * BroadcastHashJoin Inner BuildRight (30) : : : : :- * Project (28) : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * CometColumnarToRow (22) + : : : : : :- CometNativeColumnarToRow (22) : : : : : : +- CometProject (21) : : : : : : +- CometFilter (20) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -48,14 +48,14 @@ TakeOrderedAndProject (120) : : : +- BroadcastExchange (56) : : : +- * Filter (55) : : : +- * HashAggregate (54) - : : : +- * CometColumnarToRow (53) + : : : +- CometNativeColumnarToRow (53) : : : +- CometColumnarExchange (52) : : : +- * HashAggregate (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) : : : :- * Project (47) : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * CometColumnarToRow (41) + : : : : :- CometNativeColumnarToRow (41) : : : : : +- CometProject (40) : : : : : +- CometFilter (39) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -66,14 +66,14 @@ TakeOrderedAndProject (120) : : : +- ReusedExchange (48) : : +- BroadcastExchange (76) : : +- * HashAggregate (75) - : : +- * CometColumnarToRow (74) + : : +- CometNativeColumnarToRow (74) : : +- CometColumnarExchange (73) : : +- * HashAggregate (72) : : +- * Project (71) : : +- * BroadcastHashJoin Inner BuildRight (70) : : :- * Project (68) : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : :- * CometColumnarToRow (62) + : : : :- CometNativeColumnarToRow (62) : : : : +- CometProject (61) : : : : +- CometFilter (60) : : : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -85,14 +85,14 @@ TakeOrderedAndProject (120) : +- BroadcastExchange (97) : +- * Filter (96) : +- * HashAggregate (95) - : +- * CometColumnarToRow (94) + : +- CometNativeColumnarToRow (94) : +- CometColumnarExchange (93) : +- * HashAggregate (92) : +- * Project (91) : +- * BroadcastHashJoin Inner BuildRight (90) : :- * Project (88) : : +- * BroadcastHashJoin Inner BuildRight (87) - : : :- * CometColumnarToRow (82) + : : :- CometNativeColumnarToRow (82) : : : +- CometProject (81) : : : +- CometFilter (80) : : : +- CometNativeScan parquet spark_catalog.default.customer (79) @@ -103,14 +103,14 @@ TakeOrderedAndProject (120) : +- ReusedExchange (89) +- BroadcastExchange (117) +- * HashAggregate (116) - +- * CometColumnarToRow (115) + +- CometNativeColumnarToRow (115) +- CometColumnarExchange (114) +- * HashAggregate (113) +- * Project (112) +- * BroadcastHashJoin Inner BuildRight (111) :- * Project (109) : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * CometColumnarToRow (103) + : :- CometNativeColumnarToRow (103) : : +- CometProject (102) : : +- CometFilter (101) : : +- CometNativeScan parquet spark_catalog.default.customer (100) @@ -136,7 +136,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] (5) Scan parquet spark_catalog.default.store_sales @@ -158,30 +158,30 @@ Condition : isnotnull(ss_customer_sk#15) Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#15] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] (11) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#22, d_year#23] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#20] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] @@ -192,17 +192,17 @@ Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cus Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 24] +(16) CometNativeColumnarToRow Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] -(17) HashAggregate [codegen id : 24] +(17) HashAggregate [codegen id : 18] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] Aggregate Attributes [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28] Results [2]: [c_customer_id#9 AS customer_id#29, sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28 AS year_total#30] -(18) Filter [codegen id : 24] +(18) Filter [codegen id : 18] Input [2]: [customer_id#29, year_total#30] Condition : (isnotnull(year_total#30) AND (year_total#30 > 0.000000)) @@ -221,7 +221,7 @@ Condition : (isnotnull(c_customer_sk#31) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] Arguments: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14], [c_customer_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#32, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#33, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#34, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#35, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#37, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#38, 50, true, false, true) AS c_email_address#14] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [8]: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14] (23) Scan parquet spark_catalog.default.store_sales @@ -232,10 +232,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] Condition : isnotnull(ss_customer_sk#39) @@ -243,30 +243,30 @@ Condition : isnotnull(ss_customer_sk#39) Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#31] Right keys [1]: [ss_customer_sk#39] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] Input [14]: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] (29) ReusedExchange [Reuses operator id: 128] Output [2]: [d_date_sk#46, d_year#47] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#44] Right keys [1]: [d_date_sk#46] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, d_year#47] Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, d_year#47] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47] Functions [1]: [partial_sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))] @@ -277,10 +277,10 @@ Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cus Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47] Functions [1]: [sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))] @@ -291,7 +291,7 @@ Results [8]: [c_customer_id#9 AS customer_id#52, c_first_name#10 AS customer_fir Input [8]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 24] +(37) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#52] Join type: Inner @@ -312,7 +312,7 @@ Condition : (isnotnull(c_customer_sk#60) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] Arguments: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73], [c_customer_sk#60, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#61, 16, true, false, true) AS c_customer_id#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#62, 20, true, false, true) AS c_first_name#69, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#63, 30, true, false, true) AS c_last_name#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#64, 1, true, false, true) AS c_preferred_cust_flag#71, c_birth_country#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#66, 13, true, false, true) AS c_login#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#67, 50, true, false, true) AS c_email_address#73] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [8]: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73] (42) Scan parquet spark_catalog.default.catalog_sales @@ -323,10 +323,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#79), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] Condition : isnotnull(cs_bill_customer_sk#74) @@ -334,30 +334,30 @@ Condition : isnotnull(cs_bill_customer_sk#74) Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#60] Right keys [1]: [cs_bill_customer_sk#74] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] Input [14]: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] (48) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#80, d_year#81] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#79] Right keys [1]: [d_date_sk#80] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, d_year#81] Input [14]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79, d_date_sk#80, d_year#81] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, d_year#81] Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81] Functions [1]: [partial_sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))] @@ -368,17 +368,17 @@ Results [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cu Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81] Functions [1]: [sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))] Aggregate Attributes [1]: [sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))#86] Results [2]: [c_customer_id#68 AS customer_id#87, sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))#86 AS year_total#88] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#87, year_total#88] Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) @@ -386,13 +386,13 @@ Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) Input [2]: [customer_id#87, year_total#88] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 24] +(57) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#87] Join type: Inner Join condition: None -(58) Project [codegen id : 24] +(58) Project [codegen id : 18] Output [11]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#88] Input [12]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, customer_id#87, year_total#88] @@ -411,7 +411,7 @@ Condition : (isnotnull(c_customer_sk#89) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] Arguments: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73], [c_customer_sk#89, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#90, 16, true, false, true) AS c_customer_id#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#91, 20, true, false, true) AS c_first_name#69, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#92, 30, true, false, true) AS c_last_name#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#93, 1, true, false, true) AS c_preferred_cust_flag#71, c_birth_country#94, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#95, 13, true, false, true) AS c_login#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#96, 50, true, false, true) AS c_email_address#73] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [8]: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73] (63) Scan parquet spark_catalog.default.catalog_sales @@ -422,10 +422,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#102), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] Condition : isnotnull(cs_bill_customer_sk#97) @@ -433,30 +433,30 @@ Condition : isnotnull(cs_bill_customer_sk#97) Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#89] Right keys [1]: [cs_bill_customer_sk#97] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] Input [14]: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] (69) ReusedExchange [Reuses operator id: 128] Output [2]: [d_date_sk#103, d_year#104] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#102] Right keys [1]: [d_date_sk#103] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#104] Input [14]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102, d_date_sk#103, d_year#104] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#104] Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104] Functions [1]: [partial_sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))] @@ -467,10 +467,10 @@ Results [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cu Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104] Functions [1]: [sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))] @@ -481,13 +481,13 @@ Results [2]: [c_customer_id#68 AS customer_id#109, sum(((((cs_ext_list_price#101 Input [2]: [customer_id#109, year_total#110] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 24] +(77) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#109] Join type: Inner Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN (year_total#110 / year_total#88) END > CASE WHEN (year_total#30 > 0.000000) THEN (year_total#59 / year_total#30) END) -(78) Project [codegen id : 24] +(78) Project [codegen id : 18] Output [10]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110] Input [13]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#88, customer_id#109, year_total#110] @@ -506,7 +506,7 @@ Condition : (isnotnull(c_customer_sk#111) AND isnotnull(staticinvoke(class org.a Input [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] Arguments: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124], [c_customer_sk#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#112, 16, true, false, true) AS c_customer_id#119, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#113, 20, true, false, true) AS c_first_name#120, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#114, 30, true, false, true) AS c_last_name#121, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#115, 1, true, false, true) AS c_preferred_cust_flag#122, c_birth_country#116, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#117, 13, true, false, true) AS c_login#123, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#118, 50, true, false, true) AS c_email_address#124] -(82) CometColumnarToRow [codegen id : 18] +(82) CometNativeColumnarToRow Input [8]: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124] (83) Scan parquet spark_catalog.default.web_sales @@ -517,10 +517,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#130), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(84) ColumnarToRow [codegen id : 16] +(84) ColumnarToRow [codegen id : 12] Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -(85) Filter [codegen id : 16] +(85) Filter [codegen id : 12] Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] Condition : isnotnull(ws_bill_customer_sk#125) @@ -528,30 +528,30 @@ Condition : isnotnull(ws_bill_customer_sk#125) Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(87) BroadcastHashJoin [codegen id : 18] +(87) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_customer_sk#111] Right keys [1]: [ws_bill_customer_sk#125] Join type: Inner Join condition: None -(88) Project [codegen id : 18] +(88) Project [codegen id : 13] Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] Input [14]: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] (89) ReusedExchange [Reuses operator id: 124] Output [2]: [d_date_sk#131, d_year#132] -(90) BroadcastHashJoin [codegen id : 18] +(90) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#130] Right keys [1]: [d_date_sk#131] Join type: Inner Join condition: None -(91) Project [codegen id : 18] +(91) Project [codegen id : 13] Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, d_year#132] Input [14]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130, d_date_sk#131, d_year#132] -(92) HashAggregate [codegen id : 18] +(92) HashAggregate [codegen id : 13] Input [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, d_year#132] Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132] Functions [1]: [partial_sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))] @@ -562,17 +562,17 @@ Results [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] Arguments: hashpartitioning(c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(94) CometColumnarToRow [codegen id : 19] +(94) CometNativeColumnarToRow Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] -(95) HashAggregate [codegen id : 19] +(95) HashAggregate [codegen id : 14] Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132] Functions [1]: [sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))] Aggregate Attributes [1]: [sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))#137] Results [2]: [c_customer_id#119 AS customer_id#138, sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))#137 AS year_total#139] -(96) Filter [codegen id : 19] +(96) Filter [codegen id : 14] Input [2]: [customer_id#138, year_total#139] Condition : (isnotnull(year_total#139) AND (year_total#139 > 0.000000)) @@ -580,13 +580,13 @@ Condition : (isnotnull(year_total#139) AND (year_total#139 > 0.000000)) Input [2]: [customer_id#138, year_total#139] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] -(98) BroadcastHashJoin [codegen id : 24] +(98) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#138] Join type: Inner Join condition: None -(99) Project [codegen id : 24] +(99) Project [codegen id : 18] Output [11]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, year_total#139] Input [12]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, customer_id#138, year_total#139] @@ -605,7 +605,7 @@ Condition : (isnotnull(c_customer_sk#140) AND isnotnull(staticinvoke(class org.a Input [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] Arguments: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124], [c_customer_sk#140, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#141, 16, true, false, true) AS c_customer_id#119, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#142, 20, true, false, true) AS c_first_name#120, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#143, 30, true, false, true) AS c_last_name#121, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#144, 1, true, false, true) AS c_preferred_cust_flag#122, c_birth_country#145, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#146, 13, true, false, true) AS c_login#123, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#147, 50, true, false, true) AS c_email_address#124] -(103) CometColumnarToRow [codegen id : 22] +(103) CometNativeColumnarToRow Input [8]: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124] (104) Scan parquet spark_catalog.default.web_sales @@ -616,10 +616,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(105) ColumnarToRow [codegen id : 20] +(105) ColumnarToRow [codegen id : 15] Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -(106) Filter [codegen id : 20] +(106) Filter [codegen id : 15] Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] Condition : isnotnull(ws_bill_customer_sk#148) @@ -627,30 +627,30 @@ Condition : isnotnull(ws_bill_customer_sk#148) Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] -(108) BroadcastHashJoin [codegen id : 22] +(108) BroadcastHashJoin [codegen id : 16] Left keys [1]: [c_customer_sk#140] Right keys [1]: [ws_bill_customer_sk#148] Join type: Inner Join condition: None -(109) Project [codegen id : 22] +(109) Project [codegen id : 16] Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] Input [14]: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] (110) ReusedExchange [Reuses operator id: 128] Output [2]: [d_date_sk#154, d_year#155] -(111) BroadcastHashJoin [codegen id : 22] +(111) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ws_sold_date_sk#153] Right keys [1]: [d_date_sk#154] Join type: Inner Join condition: None -(112) Project [codegen id : 22] +(112) Project [codegen id : 16] Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#155] Input [14]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153, d_date_sk#154, d_year#155] -(113) HashAggregate [codegen id : 22] +(113) HashAggregate [codegen id : 16] Input [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#155] Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155] Functions [1]: [partial_sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))] @@ -661,10 +661,10 @@ Results [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] Arguments: hashpartitioning(c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(115) CometColumnarToRow [codegen id : 23] +(115) CometNativeColumnarToRow Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] -(116) HashAggregate [codegen id : 23] +(116) HashAggregate [codegen id : 17] Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155] Functions [1]: [sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))] @@ -675,13 +675,13 @@ Results [2]: [c_customer_id#119 AS customer_id#160, sum(((((ws_ext_list_price#15 Input [2]: [customer_id#160, year_total#161] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] -(118) BroadcastHashJoin [codegen id : 24] +(118) BroadcastHashJoin [codegen id : 18] Left keys [1]: [customer_id#29] Right keys [1]: [customer_id#160] Join type: Inner Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN (year_total#110 / year_total#88) END > CASE WHEN (year_total#139 > 0.000000) THEN (year_total#161 / year_total#139) END) -(119) Project [codegen id : 24] +(119) Project [codegen id : 18] Output [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] Input [13]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, year_total#139, customer_id#160, year_total#161] @@ -693,7 +693,7 @@ Arguments: 100, [customer_id#52 ASC NULLS FIRST, customer_first_name#53 ASC NULL Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometFilter (122) +- CometNativeScan parquet spark_catalog.default.date_dim (121) @@ -709,7 +709,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [2]: [d_date_sk#22, d_year#23] (124) BroadcastExchange @@ -718,7 +718,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#45 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometNativeScan parquet spark_catalog.default.date_dim (125) @@ -734,7 +734,7 @@ ReadSchema: struct Input [2]: [d_date_sk#46, d_year#47] Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#46)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#46, d_year#47] (128) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt index 54e1a1c11a..737217ff91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt @@ -10,14 +10,14 @@ TakeOrderedAndProject : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- HashAggregate - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- HashAggregate : : : : : +- Project : : : : : +- BroadcastHashJoin : : : : : :- Project : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -27,23 +27,23 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- HashAggregate - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometColumnarExchange : : : : +- HashAggregate : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- Project : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -53,24 +53,24 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -80,19 +80,19 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -102,20 +102,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -125,19 +125,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -147,7 +147,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/simplified.txt index 56bc32e144..dd6fd6a2ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - WholeStageCodegen (24) + WholeStageCodegen (18) Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] @@ -11,17 +11,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] @@ -34,69 +34,65 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter @@ -106,25 +102,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [cs_bill_customer_sk] ColumnarToRow InputAdapter @@ -134,26 +130,26 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #14 - WholeStageCodegen (19) + WholeStageCodegen (14) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) + WholeStageCodegen (13) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #16 - WholeStageCodegen (16) + WholeStageCodegen (12) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -163,25 +159,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #17 - WholeStageCodegen (23) + WholeStageCodegen (17) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) + WholeStageCodegen (16) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #19 - WholeStageCodegen (20) + WholeStageCodegen (15) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/explain.txt index cea1c1d430..979d7e593c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/explain.txt @@ -612,7 +612,7 @@ Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (111) -+- * CometColumnarToRow (110) ++- CometNativeColumnarToRow (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -628,7 +628,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(110) CometColumnarToRow [codegen id : 1] +(110) CometNativeColumnarToRow Input [2]: [d_date_sk#22, d_year#23] (111) BroadcastExchange @@ -637,7 +637,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 BroadcastExchange (115) -+- * CometColumnarToRow (114) ++- CometNativeColumnarToRow (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -653,7 +653,7 @@ ReadSchema: struct Input [2]: [d_date_sk#43, d_year#44] Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) -(114) CometColumnarToRow [codegen id : 1] +(114) CometNativeColumnarToRow Input [2]: [d_date_sk#43, d_year#44] (115) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt index 1be0816ee1..2336733a6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -47,7 +47,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/simplified.txt index 36be965b48..f984ee0702 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/simplified.txt @@ -27,11 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -51,11 +49,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index cea1c1d430..979d7e593c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -612,7 +612,7 @@ Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (111) -+- * CometColumnarToRow (110) ++- CometNativeColumnarToRow (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -628,7 +628,7 @@ ReadSchema: struct Input [2]: [d_date_sk#22, d_year#23] Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(110) CometColumnarToRow [codegen id : 1] +(110) CometNativeColumnarToRow Input [2]: [d_date_sk#22, d_year#23] (111) BroadcastExchange @@ -637,7 +637,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 BroadcastExchange (115) -+- * CometColumnarToRow (114) ++- CometNativeColumnarToRow (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -653,7 +653,7 @@ ReadSchema: struct Input [2]: [d_date_sk#43, d_year#44] Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) -(114) CometColumnarToRow [codegen id : 1] +(114) CometNativeColumnarToRow Input [2]: [d_date_sk#43, d_year#44] (115) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/extended.txt index 1be0816ee1..2336733a6a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -47,7 +47,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange 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 36be965b48..f984ee0702 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 @@ -27,11 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -51,11 +49,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/explain.txt index fe074b9df2..43157ba891 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (34) +CometNativeColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -188,14 +188,14 @@ Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -(34) CometColumnarToRow [codegen id : 2] +(34) CometNativeColumnarToRow Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometFilter (36) +- CometNativeScan parquet spark_catalog.default.date_dim (35) @@ -211,7 +211,7 @@ ReadSchema: struct Input [2]: [d_date_sk#18, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [2]: [d_date_sk#18, d_date#19] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt index d5dbd76871..f1d98d3059 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -18,7 +18,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/simplified.txt index 13760fcb0b..e28fe12099 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/simplified.txt @@ -1,45 +1,41 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - 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 [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] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #2 - WholeStageCodegen (1) - Filter [cs_warehouse_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] + 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 [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] + CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] + CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] + CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] + CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + CometColumnarExchange [cs_order_number,cs_item_sk] #2 + WholeStageCodegen (1) + Filter [cs_warehouse_sk,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] + CometExchange [cr_order_number,cr_item_sk] #4 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_state] #5 + CometProject [w_state] [w_warehouse_sk,w_state] + CometFilter [w_warehouse_sk,w_state] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/explain.txt index 00e2b8fbc1..7057990127 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -207,7 +207,7 @@ ReadSchema: struct Input [2]: [d_date_sk#18, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [2]: [d_date_sk#18, d_date#19] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt index 54a7b21486..8d2e593975 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/simplified.txt index 79e3556750..96423b87fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/simplified.txt @@ -19,11 +19,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] CometExchange [cr_order_number,cr_item_sk] #4 CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt index 00e2b8fbc1..7057990127 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt @@ -191,7 +191,7 @@ Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -207,7 +207,7 @@ ReadSchema: struct Input [2]: [d_date_sk#18, d_date#19] Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [2]: [d_date_sk#18, d_date#19] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/extended.txt index 54a7b21486..8d2e593975 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort 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 79e3556750..96423b87fb 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 @@ -19,11 +19,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] CometExchange [cr_order_number,cr_item_sk] #4 CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/explain.txt index cbce1b2b45..50c83dd2b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -102,6 +102,6 @@ Functions: [] Input [1]: [i_product_name#11] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#11 ASC NULLS FIRST], output=[i_product_name#11]), [i_product_name#11], 100, 0, [i_product_name#11 ASC NULLS FIRST], [i_product_name#11] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [1]: [i_product_name#11] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/extended.txt index b40ebddb6f..6f7d077cf3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/simplified.txt index 2557f51684..76e150734c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometExchange [i_product_name] #1 CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + CometProject [i_product_name] [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometNativeScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] + CometExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/explain.txt index 3b1ad984a3..d562e8b7ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -102,6 +102,6 @@ Functions: [] Input [1]: [i_product_name#11] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#11 ASC NULLS FIRST], output=[i_product_name#11]), [i_product_name#11], 100, 0, [i_product_name#11 ASC NULLS FIRST], [i_product_name#11] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [1]: [i_product_name#11] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/extended.txt index 9d802b5033..a3c8dd9568 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/simplified.txt index 99c935aaa4..586ccd2852 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometExchange [i_product_name] #1 CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + CometProject [i_product_name] [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] + CometExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 3b1ad984a3..d562e8b7ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -102,6 +102,6 @@ Functions: [] Input [1]: [i_product_name#11] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#11 ASC NULLS FIRST], output=[i_product_name#11]), [i_product_name#11], 100, 0, [i_product_name#11 ASC NULLS FIRST], [i_product_name#11] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [1]: [i_product_name#11] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/extended.txt index 9d802b5033..a3c8dd9568 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange 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 99c935aaa4..586ccd2852 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_product_name] + CometHashAggregate [i_product_name] + CometExchange [i_product_name] #1 CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] + CometProject [i_product_name] [i_product_name] + CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + CometBroadcastExchange [i_manufact] #2 + CometProject [i_manufact] + CometFilter [item_cnt,i_manufact] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] + CometExchange [i_manufact] #3 + CometHashAggregate [i_manufact,count] + CometProject [i_manufact] [i_manufact] + CometFilter [i_category,i_manufact,i_size,i_color,i_units] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/explain.txt index af01446009..4c0c21880d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/simplified.txt index 96ab34454d..6ad3968250 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(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 [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] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(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 [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] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 + CometProject [i_category] [i_item_sk,i_category_id,i_category] + CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/explain.txt index 261e6ddbaa..4172904405 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_s Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/extended.txt index 5eaea983af..cebd2e28d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/simplified.txt index c7064b3ad8..d40373ef42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -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 [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 [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] - CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] +CometNativeColumnarToRow + CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + 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 [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] + CometBroadcastHashJoin [d_date_sk,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [d_day_name] [d_date_sk,d_day_name] + CometFilter [d_date_sk,d_year,d_day_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] + CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/explain.txt index e66d71b41e..7669ac24b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/explain.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject (34) : : : +- * Project (13) : : : +- * Filter (12) : : : +- Window (11) - : : : +- * CometColumnarToRow (10) + : : : +- CometNativeColumnarToRow (10) : : : +- CometSort (9) : : : +- CometExchange (8) : : : +- CometFilter (7) @@ -24,11 +24,11 @@ TakeOrderedAndProject (34) : : +- * Project (20) : : +- * Filter (19) : : +- Window (18) - : : +- * CometColumnarToRow (17) + : : +- CometNativeColumnarToRow (17) : : +- CometSort (16) : : +- ReusedExchange (15) : +- BroadcastExchange (28) - : +- * CometColumnarToRow (27) + : +- CometNativeColumnarToRow (27) : +- CometProject (26) : +- CometFilter (25) : +- CometNativeScan parquet spark_catalog.default.item (24) @@ -76,22 +76,22 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(10) CometColumnarToRow [codegen id : 1] +(10) CometNativeColumnarToRow Input [2]: [item_sk#7, rank_col#8] (11) Window Input [2]: [item_sk#7, rank_col#8] Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] -(12) Filter [codegen id : 2] +(12) Filter [codegen id : 1] Input [3]: [item_sk#7, rank_col#8, rnk#11] Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(13) Project [codegen id : 2] +(13) Project [codegen id : 1] Output [2]: [item_sk#7, rnk#11] Input [3]: [item_sk#7, rank_col#8, rnk#11] -(14) Sort [codegen id : 2] +(14) Sort [codegen id : 1] Input [2]: [item_sk#7, rnk#11] Arguments: [rnk#11 ASC NULLS FIRST], false, 0 @@ -102,32 +102,32 @@ Output [2]: [item_sk#12, rank_col#13] Input [2]: [item_sk#12, rank_col#13] Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [item_sk#12, rank_col#13] (18) Window Input [2]: [item_sk#12, rank_col#13] Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] -(19) Filter [codegen id : 4] +(19) Filter [codegen id : 2] Input [3]: [item_sk#12, rank_col#13, rnk#14] Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) -(20) Project [codegen id : 4] +(20) Project [codegen id : 2] Output [2]: [item_sk#12, rnk#14] Input [3]: [item_sk#12, rank_col#13, rnk#14] -(21) Sort [codegen id : 4] +(21) Sort [codegen id : 2] Input [2]: [item_sk#12, rnk#14] Arguments: [rnk#14 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 7] +(22) SortMergeJoin [codegen id : 3] Left keys [1]: [rnk#11] Right keys [1]: [rnk#14] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 3] Output [3]: [item_sk#7, rnk#11, item_sk#12] Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] @@ -146,33 +146,33 @@ Condition : isnotnull(i_item_sk#15) Input [2]: [i_item_sk#15, i_product_name#16] Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] -(27) CometColumnarToRow [codegen id : 5] +(27) CometNativeColumnarToRow Input [2]: [i_item_sk#15, i_product_name#17] (28) BroadcastExchange Input [2]: [i_item_sk#15, i_product_name#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 3] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(30) Project [codegen id : 7] +(30) Project [codegen id : 3] Output [3]: [rnk#11, item_sk#12, i_product_name#17] Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] (31) ReusedExchange [Reuses operator id: 28] Output [2]: [i_item_sk#18, i_product_name#19] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 3] Left keys [1]: [item_sk#12] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 3] Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] @@ -183,7 +183,7 @@ Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_per ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (41) +CometNativeColumnarToRow (41) +- CometHashAggregate (40) +- CometExchange (39) +- CometHashAggregate (38) @@ -221,7 +221,7 @@ Input [3]: [ss_store_sk#23, sum#26, count#27] Keys [1]: [ss_store_sk#23] Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [rank_col#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt index edc1cd324f..d2f31a9b37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt @@ -9,12 +9,12 @@ TakeOrderedAndProject : : : +- Project : : : +- Filter : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometSort : : : +- CometExchange : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -31,12 +31,12 @@ TakeOrderedAndProject : : +- Project : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometExchange : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/simplified.txt index 6ff27fc6c4..e71eb7d62b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) + WholeStageCodegen (3) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -7,53 +7,45 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Project [item_sk,rnk,item_sk] SortMergeJoin [rnk,rnk] InputAdapter - WholeStageCodegen (2) + WholeStageCodegen (1) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometExchange #1 - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometNativeColumnarToRow + CometSort [item_sk,rank_col] + CometExchange #1 + CometFilter [item_sk,rank_col] + Subquery #1 + CometNativeColumnarToRow + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_store_sk] #3 + 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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_item_sk] #2 + 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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] InputAdapter - WholeStageCodegen (4) + WholeStageCodegen (2) Sort [rnk] Project [item_sk,rnk] Filter [rnk,item_sk] InputAdapter Window [rank_col] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - ReusedExchange [item_sk,rank_col] #1 + CometNativeColumnarToRow + CometSort [item_sk,rank_col] + ReusedExchange [item_sk,rank_col] #1 InputAdapter BroadcastExchange #4 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_product_name] + CometNativeColumnarToRow + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt index 134ab7939e..6ac3d5bbe2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject (34) : : +- CometSort (16) : : +- ReusedExchange (15) : +- BroadcastExchange (28) - : +- * CometColumnarToRow (27) + : +- CometNativeColumnarToRow (27) : +- CometProject (26) : +- CometFilter (25) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) @@ -121,13 +121,13 @@ Input [3]: [item_sk#12, rank_col#13, rnk#14] Input [2]: [item_sk#12, rnk#14] Arguments: [rnk#14 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 7] +(22) SortMergeJoin [codegen id : 5] Left keys [1]: [rnk#11] Right keys [1]: [rnk#14] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 5] Output [3]: [item_sk#7, rnk#11, item_sk#12] Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] @@ -146,33 +146,33 @@ Condition : isnotnull(i_item_sk#15) Input [2]: [i_item_sk#15, i_product_name#16] Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] -(27) CometColumnarToRow [codegen id : 5] +(27) CometNativeColumnarToRow Input [2]: [i_item_sk#15, i_product_name#17] (28) BroadcastExchange Input [2]: [i_item_sk#15, i_product_name#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 5] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(30) Project [codegen id : 7] +(30) Project [codegen id : 5] Output [3]: [rnk#11, item_sk#12, i_product_name#17] Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] (31) ReusedExchange [Reuses operator id: 28] Output [2]: [i_item_sk#18, i_product_name#19] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 5] Left keys [1]: [item_sk#12] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 5] Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt index 5ef0fb3b87..0952fbf8d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt index e3ec416397..36dfa432dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) + WholeStageCodegen (5) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -49,11 +49,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ReusedExchange [item_sk,rank_col] #1 InputAdapter BroadcastExchange #4 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] + CometNativeColumnarToRow + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index 134ab7939e..6ac3d5bbe2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject (34) : : +- CometSort (16) : : +- ReusedExchange (15) : +- BroadcastExchange (28) - : +- * CometColumnarToRow (27) + : +- CometNativeColumnarToRow (27) : +- CometProject (26) : +- CometFilter (25) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) @@ -121,13 +121,13 @@ Input [3]: [item_sk#12, rank_col#13, rnk#14] Input [2]: [item_sk#12, rnk#14] Arguments: [rnk#14 ASC NULLS FIRST], false, 0 -(22) SortMergeJoin [codegen id : 7] +(22) SortMergeJoin [codegen id : 5] Left keys [1]: [rnk#11] Right keys [1]: [rnk#14] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 5] Output [3]: [item_sk#7, rnk#11, item_sk#12] Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] @@ -146,33 +146,33 @@ Condition : isnotnull(i_item_sk#15) Input [2]: [i_item_sk#15, i_product_name#16] Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] -(27) CometColumnarToRow [codegen id : 5] +(27) CometNativeColumnarToRow Input [2]: [i_item_sk#15, i_product_name#17] (28) BroadcastExchange Input [2]: [i_item_sk#15, i_product_name#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(29) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 5] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(30) Project [codegen id : 7] +(30) Project [codegen id : 5] Output [3]: [rnk#11, item_sk#12, i_product_name#17] Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] (31) ReusedExchange [Reuses operator id: 28] Output [2]: [i_item_sk#18, i_product_name#19] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 5] Left keys [1]: [item_sk#12] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 5] Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt index 5ef0fb3b87..0952fbf8d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item 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 e3ec416397..36dfa432dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) + WholeStageCodegen (5) Project [rnk,i_product_name,i_product_name] BroadcastHashJoin [item_sk,i_item_sk] Project [rnk,item_sk,i_product_name] @@ -49,11 +49,9 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] ReusedExchange [item_sk,rank_col] #1 InputAdapter BroadcastExchange #4 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] + CometNativeColumnarToRow + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] InputAdapter ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/explain.txt index 1d20f0a3f0..0f29d94fd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * Project (34) @@ -19,22 +19,22 @@ TakeOrderedAndProject (39) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.web_sales (1) : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) + : : : : +- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.customer (4) : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometProject (12) : : : +- CometFilter (11) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) : : +- ReusedExchange (17) : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometProject (22) : +- CometFilter (21) : +- CometNativeScan parquet spark_catalog.default.item (20) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.item (27) @@ -48,10 +48,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 1] Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) @@ -66,20 +66,20 @@ ReadSchema: struct Input [2]: [c_customer_sk#7, c_current_addr_sk#8] Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [2]: [c_customer_sk#7, c_current_addr_sk#8] (7) BroadcastExchange Input [2]: [c_customer_sk#7, c_current_addr_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 6] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_bill_customer_sk#3] Right keys [1]: [c_customer_sk#7] Join type: Inner Join condition: None -(9) Project [codegen id : 6] +(9) Project [codegen id : 1] Output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] @@ -98,33 +98,33 @@ Condition : isnotnull(ca_address_sk#9) Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#11, 10, true, false, true) AS ca_zip#12] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] (14) BroadcastExchange Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#8] Right keys [1]: [ca_address_sk#9] Join type: Inner Join condition: None -(16) Project [codegen id : 6] +(16) Project [codegen id : 1] Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] (17) ReusedExchange [Reuses operator id: 44] Output [1]: [d_date_sk#13] -(18) BroadcastHashJoin [codegen id : 6] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#5] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(19) Project [codegen id : 1] Output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] @@ -143,20 +143,20 @@ Condition : isnotnull(i_item_sk#14) Input [2]: [i_item_sk#14, i_item_id#15] Arguments: [i_item_sk#14, i_item_id#16], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#16] -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [i_item_sk#14, i_item_id#16] (24) BroadcastExchange Input [2]: [i_item_sk#14, i_item_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#2] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(26) Project [codegen id : 1] Output [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16] Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#14, i_item_id#16] @@ -175,28 +175,28 @@ Condition : i_item_sk#14 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#14, i_item_id#15] Arguments: [i_item_id#17], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#17] -(30) CometColumnarToRow [codegen id : 5] +(30) CometNativeColumnarToRow Input [1]: [i_item_id#17] (31) BroadcastExchange Input [1]: [i_item_id#17] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 6] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [i_item_id#16] Right keys [1]: [i_item_id#17] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 6] +(33) Filter [codegen id : 1] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(34) Project [codegen id : 6] +(34) Project [codegen id : 1] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] -(35) HashAggregate [codegen id : 6] +(35) HashAggregate [codegen id : 1] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] @@ -207,10 +207,10 @@ Results [3]: [ca_zip#12, ca_city#10, sum#19] Input [3]: [ca_zip#12, ca_city#10, sum#19] Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 7] +(37) CometNativeColumnarToRow Input [3]: [ca_zip#12, ca_city#10, sum#19] -(38) HashAggregate [codegen id : 7] +(38) HashAggregate [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#19] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] @@ -225,7 +225,7 @@ Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometNativeScan parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2) Input [3]: [d_date_sk#13, d_year#22, d_qoy#23] Arguments: [d_date_sk#13], [d_date_sk#13] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt index c58a64e326..5b686ba177 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -19,31 +19,31 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/simplified.txt index 81ecde84b2..7a0bb62bd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) + WholeStageCodegen (1) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] @@ -23,42 +23,32 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + CometNativeColumnarToRow + CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_city,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/explain.txt index 9d3cf3d382..332f0bdacf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/explain.txt @@ -34,7 +34,7 @@ TakeOrderedAndProject (39) : +- CometFilter (21) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) @@ -160,7 +160,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] -(26) CometColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 1] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -178,28 +178,28 @@ Condition : i_item_sk#16 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#16, i_item_id#17] Arguments: [i_item_id#19], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#19] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [i_item_id#19] (31) BroadcastExchange Input [1]: [i_item_id#19] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] -(32) BroadcastHashJoin [codegen id : 2] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [i_item_id#18] Right keys [1]: [i_item_id#19] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 2] +(33) Filter [codegen id : 1] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(34) Project [codegen id : 2] +(34) Project [codegen id : 1] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -(35) HashAggregate [codegen id : 2] +(35) HashAggregate [codegen id : 1] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] @@ -210,10 +210,10 @@ Results [3]: [ca_zip#12, ca_city#10, sum#21] Input [3]: [ca_zip#12, ca_city#10, sum#21] Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(37) CometColumnarToRow [codegen id : 3] +(37) CometColumnarToRow [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#21] -(38) HashAggregate [codegen id : 3] +(38) HashAggregate [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#21] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] @@ -228,7 +228,7 @@ Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -249,7 +249,7 @@ Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2) Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt index 4a57091056..23662b8aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/simplified.txt index d78fc716ab..2d2ba0c558 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] @@ -23,12 +23,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] @@ -46,9 +44,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [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] + CometNativeColumnarToRow + CometProject [i_item_id] [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index 9d3cf3d382..332f0bdacf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -34,7 +34,7 @@ TakeOrderedAndProject (39) : +- CometFilter (21) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) @@ -160,7 +160,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] -(26) CometColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 1] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] (27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -178,28 +178,28 @@ Condition : i_item_sk#16 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#16, i_item_id#17] Arguments: [i_item_id#19], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#19] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [i_item_id#19] (31) BroadcastExchange Input [1]: [i_item_id#19] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] -(32) BroadcastHashJoin [codegen id : 2] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [i_item_id#18] Right keys [1]: [i_item_id#19] Join type: ExistenceJoin(exists#1) Join condition: None -(33) Filter [codegen id : 2] +(33) Filter [codegen id : 1] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) -(34) Project [codegen id : 2] +(34) Project [codegen id : 1] Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -(35) HashAggregate [codegen id : 2] +(35) HashAggregate [codegen id : 1] Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] @@ -210,10 +210,10 @@ Results [3]: [ca_zip#12, ca_city#10, sum#21] Input [3]: [ca_zip#12, ca_city#10, sum#21] Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(37) CometColumnarToRow [codegen id : 3] +(37) CometColumnarToRow [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#21] -(38) HashAggregate [codegen id : 3] +(38) HashAggregate [codegen id : 2] Input [3]: [ca_zip#12, ca_city#10, sum#21] Keys [2]: [ca_zip#12, ca_city#10] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] @@ -228,7 +228,7 @@ Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -249,7 +249,7 @@ Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2) Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt index 4a57091056..23662b8aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,7 +39,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt index d78fc716ab..2d2ba0c558 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] Project [ws_sales_price,ca_city,ca_zip] Filter [ca_zip,exists] @@ -23,12 +23,10 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 CometFilter [c_customer_sk,c_current_addr_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] @@ -46,9 +44,7 @@ TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [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] + CometNativeColumnarToRow + CometProject [i_item_id] [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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/explain.txt index 994cd12469..40ba9461cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (41) :- * Project (37) : +- * BroadcastHashJoin Inner BuildRight (36) : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) + : : +- CometNativeColumnarToRow (29) : : +- CometColumnarExchange (28) : : +- * HashAggregate (27) : : +- * Project (26) @@ -21,21 +21,21 @@ TakeOrderedAndProject (41) : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : +- ReusedExchange (4) : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) + : : : : +- CometNativeColumnarToRow (10) : : : : +- CometProject (9) : : : : +- CometFilter (8) : : : : +- CometNativeScan parquet spark_catalog.default.store (7) : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) + : : : +- CometNativeColumnarToRow (17) : : : +- CometProject (16) : : : +- CometFilter (15) : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometFilter (22) : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) + : +- CometNativeColumnarToRow (34) : +- CometProject (33) : +- CometFilter (32) : +- CometNativeScan parquet spark_catalog.default.customer (31) @@ -50,23 +50,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#10] -(5) BroadcastHashJoin [codegen id : 5] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(6) Project [codegen id : 5] +(6) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] @@ -85,20 +85,20 @@ Condition : (s_city#12 IN (Fairview,Midway) AND isnotnull(s_store_sk#11)) Input [2]: [s_store_sk#11, s_city#12] Arguments: [s_store_sk#11], [s_store_sk#11] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#11] (11) BroadcastExchange Input [1]: [s_store_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11] @@ -117,20 +117,20 @@ Condition : (((hd_dep_count#14 = 4) OR (hd_vehicle_count#15 = 3)) AND isnotnull( Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] Arguments: [hd_demo_sk#13], [hd_demo_sk#13] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#13] (18) BroadcastExchange Input [1]: [hd_demo_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#13] @@ -145,24 +145,24 @@ ReadSchema: struct Input [2]: [ca_address_sk#16, ca_city#17] Condition : (isnotnull(ca_address_sk#16) AND isnotnull(ca_city#17)) -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [ca_address_sk#16, ca_city#17] (24) BroadcastExchange Input [2]: [ca_address_sk#16, ca_city#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(26) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#16, ca_city#17] -(27) HashAggregate [codegen id : 5] +(27) HashAggregate [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] @@ -173,10 +173,10 @@ Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, su Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometColumnarToRow [codegen id : 8] +(29) CometNativeColumnarToRow Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -(30) HashAggregate [codegen id : 8] +(30) HashAggregate [codegen id : 2] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] @@ -198,33 +198,33 @@ Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] Arguments: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32], [c_customer_sk#27, c_current_addr_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#32] -(34) CometColumnarToRow [codegen id : 6] +(34) CometNativeColumnarToRow Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] (35) BroadcastExchange Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#27] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 2] Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32] Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] (38) ReusedExchange [Reuses operator id: 24] Output [2]: [ca_address_sk#33, ca_city#34] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_current_addr_sk#28] Right keys [1]: [ca_address_sk#33] Join type: Inner Join condition: NOT (ca_city#34 = bought_city#24) -(40) Project [codegen id : 8] +(40) Project [codegen id : 2] Output [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] @@ -236,7 +236,7 @@ Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometNativeScan parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((d_dow#36 IN (6,0) AND d_year#35 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#35, d_dow#36] Arguments: [d_date_sk#10], [d_date_sk#10] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt index c122bf3803..cebcd85144 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -20,36 +20,36 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/simplified.txt index b4b78689b1..2b3e7984ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -24,44 +24,34 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometNativeColumnarToRow + CometFilter [ca_address_sk,ca_city] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/explain.txt index ca7ae9df0c..044048fc80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/explain.txt @@ -225,7 +225,7 @@ Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/simplified.txt index e646cb620a..59123d0f6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index ca7ae9df0c..044048fc80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -225,7 +225,7 @@ Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e646cb620a..59123d0f6f 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/explain.txt index 3f98e1bd21..0cf411ed0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.store_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.store (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.store_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#6] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] (17) BroadcastExchange Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#7] Right keys [1]: [s_store_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] @@ -151,10 +151,10 @@ Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_yea Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] (27) Window Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] (32) ReusedExchange [Reuses operator id: 21] Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_ Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] (38) Window Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] @@ -228,13 +228,13 @@ Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year# Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] @@ -245,14 +245,14 @@ Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] (46) Window Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] @@ -260,13 +260,13 @@ Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year# Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt index 092083e465..ab44bbef5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/simplified.txt index 017805e5b2..139a34baee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk,s_store_name,s_company_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt index 1c039203ce..dce7dc0840 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt index 19124a9e20..56fdda3162 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index 1c039203ce..dce7dc0840 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 19124a9e20..56fdda3162 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/explain.txt index 494c1d4338..a582c713ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * HashAggregate (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometColumnarExchange (28) +- * HashAggregate (27) +- * Project (26) @@ -15,16 +15,16 @@ : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) + : : : +- CometNativeColumnarToRow (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.store (4) : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) + : : +- CometNativeColumnarToRow (13) : : +- CometProject (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (10) : +- BroadcastExchange (21) - : +- * CometColumnarToRow (20) + : +- CometNativeColumnarToRow (20) : +- CometProject (19) : +- CometFilter (18) : +- CometNativeScan parquet spark_catalog.default.customer_address (17) @@ -39,10 +39,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) @@ -57,20 +57,20 @@ ReadSchema: struct Input [1]: [s_store_sk#9] Condition : isnotnull(s_store_sk#9) -(6) CometColumnarToRow [codegen id : 1] +(6) CometNativeColumnarToRow Input [1]: [s_store_sk#9] (7) BroadcastExchange Input [1]: [s_store_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(8) BroadcastHashJoin [codegen id : 5] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#9] Join type: Inner Join condition: None -(9) Project [codegen id : 5] +(9) Project [codegen id : 1] Output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] @@ -89,20 +89,20 @@ Condition : (isnotnull(cd_demo_sk#10) AND ((((staticinvoke(class org.apache.spar Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) AS cd_marital_status#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) AS cd_education_status#14] -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] (14) BroadcastExchange Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 5] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) -(16) Project [codegen id : 5] +(16) Project [codegen id : 1] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] @@ -121,37 +121,37 @@ Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#18] -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [2]: [ca_address_sk#15, ca_state#18] (21) BroadcastExchange Input [2]: [ca_address_sk#15, ca_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 5] +(22) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#15] Join type: Inner Join condition: ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))) -(23) Project [codegen id : 5] +(23) Project [codegen id : 1] Output [2]: [ss_quantity#4, ss_sold_date_sk#7] Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] (24) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#19] -(25) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(26) Project [codegen id : 1] Output [1]: [ss_quantity#4] Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] -(27) HashAggregate [codegen id : 5] +(27) HashAggregate [codegen id : 1] Input [1]: [ss_quantity#4] Keys: [] Functions [1]: [partial_sum(ss_quantity#4)] @@ -162,10 +162,10 @@ Results [1]: [sum#21] Input [1]: [sum#21] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometColumnarToRow [codegen id : 6] +(29) CometNativeColumnarToRow Input [1]: [sum#21] -(30) HashAggregate [codegen id : 6] +(30) HashAggregate [codegen id : 2] Input [1]: [sum#21] Keys: [] Functions [1]: [sum(ss_quantity#4)] @@ -176,7 +176,7 @@ Results [1]: [sum(ss_quantity#4)#22 AS sum(ss_quantity)#23] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#24) AND (d_year#24 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#19, d_year#24] Arguments: [d_date_sk#19], [d_date_sk#19] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt index 1ef8426231..5e6a26863a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,26 +15,26 @@ HashAggregate : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/simplified.txt index 0cd7e900b0..b258895829 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (6) +WholeStageCodegen (2) HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_quantity] [sum,sum] Project [ss_quantity] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -19,34 +19,26 @@ WholeStageCodegen (6) Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/explain.txt index 71924d1e6d..3afd893173 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/explain.txt @@ -171,7 +171,7 @@ Input [1]: [sum(ss_quantity)#22] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -192,7 +192,7 @@ Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#19, d_year#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt index 57fbc2d035..347702b724 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/simplified.txt index e29e101c7e..52cdd88b76 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/simplified.txt @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index 71924d1e6d..3afd893173 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -171,7 +171,7 @@ Input [1]: [sum(ss_quantity)#22] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -192,7 +192,7 @@ Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#19, d_year#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#19] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/extended.txt index 57fbc2d035..347702b724 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e29e101c7e..52cdd88b76 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 @@ -16,12 +16,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/explain.txt index 0a43ebc390..90cb3f712c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (84) +CometNativeColumnarToRow (84) +- CometTakeOrderedAndProject (83) +- CometHashAggregate (82) +- CometColumnarExchange (81) @@ -10,11 +10,11 @@ : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometSort (20) : +- CometColumnarExchange (19) : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometColumnarExchange (16) : +- * HashAggregate (15) : +- * Project (14) @@ -26,7 +26,7 @@ : : : +- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) + : : +- CometNativeColumnarToRow (9) : : +- CometProject (8) : : +- CometFilter (7) : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) @@ -36,11 +36,11 @@ : +- Window (50) : +- * Sort (49) : +- Window (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometSort (46) : +- CometColumnarExchange (45) : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -52,7 +52,7 @@ : : : +- * Filter (29) : : : +- * ColumnarToRow (28) : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) + : : +- CometNativeColumnarToRow (35) : : +- CometProject (34) : : +- CometFilter (33) : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) @@ -62,11 +62,11 @@ +- Window (76) +- * Sort (75) +- Window (74) - +- * CometColumnarToRow (73) + +- CometNativeColumnarToRow (73) +- CometSort (72) +- CometColumnarExchange (71) +- * HashAggregate (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometColumnarExchange (68) +- * HashAggregate (67) +- * Project (66) @@ -78,7 +78,7 @@ : : +- * Filter (55) : : +- * ColumnarToRow (54) : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) + : +- CometNativeColumnarToRow (61) : +- CometProject (60) : +- CometFilter (59) : +- CometNativeScan parquet spark_catalog.default.store_returns (58) @@ -123,33 +123,33 @@ Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AN Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometColumnarToRow +(9) CometNativeColumnarToRow Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [2]: [ws_order_number#2, ws_item_sk#1] Right keys [2]: [wr_order_number#9, wr_item_sk#8] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] (12) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#13] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#6] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 2] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -160,10 +160,10 @@ Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty# Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -178,14 +178,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 5] +(21) CometNativeColumnarToRow Input [3]: [item#30, return_ratio#31, currency_ratio#32] (22) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 4] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 @@ -193,11 +193,11 @@ Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 5] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(26) Project [codegen id : 7] +(26) Project [codegen id : 5] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -209,14 +209,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -(29) Filter [codegen id : 8] +(29) Filter [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(30) Project [codegen id : 8] +(30) Project [codegen id : 6] Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] @@ -239,33 +239,33 @@ Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.0 Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(35) CometColumnarToRow +(35) CometNativeColumnarToRow Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 7] Left keys [2]: [cs_order_number#37, cs_item_sk#36] Right keys [2]: [cr_order_number#43, cr_item_sk#42] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 7] Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] (38) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#47] -(39) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#47] Join type: Inner Join condition: None -(40) Project [codegen id : 10] +(40) Project [codegen id : 7] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] -(41) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -276,10 +276,10 @@ Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometColumnarToRow [codegen id : 11] +(43) CometNativeColumnarToRow Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 8] Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -294,14 +294,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 12] +(47) CometNativeColumnarToRow Input [3]: [item#64, return_ratio#65, currency_ratio#66] (48) Window Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] -(49) Sort [codegen id : 13] +(49) Sort [codegen id : 9] Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 @@ -309,11 +309,11 @@ Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] -(51) Filter [codegen id : 14] +(51) Filter [codegen id : 10] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) -(52) Project [codegen id : 14] +(52) Project [codegen id : 10] Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] @@ -325,14 +325,14 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 15] +(54) ColumnarToRow [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -(55) Filter [codegen id : 15] +(55) Filter [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) -(56) Project [codegen id : 15] +(56) Project [codegen id : 11] Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] @@ -355,33 +355,33 @@ Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AN Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(61) CometColumnarToRow +(61) CometNativeColumnarToRow Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(62) BroadcastHashJoin [codegen id : 17] +(62) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(63) Project [codegen id : 12] Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] (64) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#81] -(65) BroadcastHashJoin [codegen id : 17] +(65) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ss_sold_date_sk#75] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(66) Project [codegen id : 12] Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] -(67) HashAggregate [codegen id : 17] +(67) HashAggregate [codegen id : 12] Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Keys [1]: [ss_item_sk#70] Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -392,10 +392,10 @@ Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(69) CometColumnarToRow [codegen id : 18] +(69) CometNativeColumnarToRow Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -(70) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 13] Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Keys [1]: [ss_item_sk#70] Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -410,14 +410,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [3]: [item#98, return_ratio#99, currency_ratio#100] (74) Window Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] -(75) Sort [codegen id : 20] +(75) Sort [codegen id : 14] Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 @@ -425,17 +425,17 @@ Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] -(77) Filter [codegen id : 21] +(77) Filter [codegen id : 15] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) -(78) Project [codegen id : 21] +(78) Project [codegen id : 15] Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] (79) Union -(80) HashAggregate [codegen id : 22] +(80) HashAggregate [codegen id : 16] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] @@ -455,14 +455,14 @@ Functions: [] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) CometColumnarToRow [codegen id : 23] +(84) CometNativeColumnarToRow Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometNativeScan parquet spark_catalog.default.date_dim (85) @@ -483,7 +483,7 @@ Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 Input [3]: [d_date_sk#13, d_year#104, d_moy#105] Arguments: [d_date_sk#13], [d_date_sk#13] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index 6c4ded1515..0090739d55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -9,11 +9,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -27,16 +27,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -45,11 +45,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -62,12 +62,12 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -76,11 +76,11 @@ CometColumnarToRow +- Window +- Sort +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -93,12 +93,12 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/simplified.txt index 047269434d..018aec1d47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/simplified.txt @@ -1,140 +1,130 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + CometNativeColumnarToRow + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #6 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [cs_item_sk] #7 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (15) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #9 + WholeStageCodegen (13) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (12) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (11) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt index 338f5f4201..0ecba1505f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt index ca80833ee5..3353e1c331 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 338f5f4201..0ecba1505f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ca80833ee5..3353e1c331 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 @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/explain.txt index a0a95bd28b..c0d72cc899 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/explain.txt @@ -1,13 +1,13 @@ == Physical Plan == TakeOrderedAndProject (81) +- * HashAggregate (80) - +- * CometColumnarToRow (79) + +- CometNativeColumnarToRow (79) +- CometColumnarExchange (78) +- * HashAggregate (77) +- * Expand (76) +- Union (75) :- * HashAggregate (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometColumnarExchange (21) : +- * HashAggregate (20) : +- * Project (19) @@ -25,12 +25,12 @@ TakeOrderedAndProject (81) : : : +- Scan parquet spark_catalog.default.store_returns (5) : : +- ReusedExchange (10) : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometProject (15) : +- CometFilter (14) : +- CometNativeScan parquet spark_catalog.default.store (13) :- * HashAggregate (46) - : +- * CometColumnarToRow (45) + : +- CometNativeColumnarToRow (45) : +- CometColumnarExchange (44) : +- * HashAggregate (43) : +- * Project (42) @@ -48,12 +48,12 @@ TakeOrderedAndProject (81) : : : +- Scan parquet spark_catalog.default.catalog_returns (28) : : +- ReusedExchange (33) : +- BroadcastExchange (40) - : +- * CometColumnarToRow (39) + : +- CometNativeColumnarToRow (39) : +- CometProject (38) : +- CometFilter (37) : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) +- * HashAggregate (74) - +- * CometColumnarToRow (73) + +- CometNativeColumnarToRow (73) +- CometColumnarExchange (72) +- * HashAggregate (71) +- * Project (70) @@ -70,13 +70,13 @@ TakeOrderedAndProject (81) : : :- BroadcastExchange (53) : : : +- * ColumnarToRow (52) : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : +- * CometColumnarToRow (57) + : : +- CometNativeColumnarToRow (57) : : +- CometProject (56) : : +- CometFilter (55) : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) : +- ReusedExchange (61) +- BroadcastExchange (68) - +- * CometColumnarToRow (67) + +- CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometNativeScan parquet spark_catalog.default.web_site (64) @@ -125,13 +125,13 @@ Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_s (10) ReusedExchange [Reuses operator id: 86] Output [1]: [d_date_sk#22] -(11) BroadcastHashJoin [codegen id : 5] +(11) BroadcastHashJoin [codegen id : 3] Left keys [1]: [date_sk#7] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(12) Project [codegen id : 5] +(12) Project [codegen id : 3] Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] @@ -150,24 +150,24 @@ Condition : isnotnull(s_store_sk#23) Input [2]: [s_store_sk#23, s_store_id#24] Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#23, s_store_id#25] (17) BroadcastExchange Input [2]: [s_store_sk#23, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 5] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [store_sk#6] Right keys [1]: [s_store_sk#23] Join type: Inner Join condition: None -(19) Project [codegen id : 5] +(19) Project [codegen id : 3] Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] -(20) HashAggregate [codegen id : 5] +(20) HashAggregate [codegen id : 3] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] @@ -178,10 +178,10 @@ Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(23) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 4] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -196,14 +196,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] +(25) ColumnarToRow [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -(26) Filter [codegen id : 7] +(26) Filter [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Condition : isnotnull(cs_catalog_page_sk#43) -(27) Project [codegen id : 7] +(27) Project [codegen id : 5] Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] @@ -215,14 +215,14 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 8] +(29) ColumnarToRow [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -(30) Filter [codegen id : 8] +(30) Filter [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(31) Project [codegen id : 8] +(31) Project [codegen id : 6] Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] @@ -231,13 +231,13 @@ Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_retur (33) ReusedExchange [Reuses operator id: 86] Output [1]: [d_date_sk#63] -(34) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(35) Project [codegen id : 11] +(35) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -256,24 +256,24 @@ Condition : isnotnull(cp_catalog_page_sk#64) Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#65, 16, true, false, true) AS cp_catalog_page_id#66] -(39) CometColumnarToRow [codegen id : 10] +(39) CometNativeColumnarToRow Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] (40) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(41) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(42) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] @@ -284,10 +284,10 @@ Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(45) CometColumnarToRow [codegen id : 12] +(45) CometNativeColumnarToRow Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -(46) HashAggregate [codegen id : 12] +(46) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -302,14 +302,14 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 13] +(48) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(49) Filter [codegen id : 13] +(49) Filter [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Condition : isnotnull(ws_web_site_sk#84) -(50) Project [codegen id : 13] +(50) Project [codegen id : 9] Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] @@ -320,7 +320,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 14] +(52) ColumnarToRow [codegen id : 10] Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (53) BroadcastExchange @@ -342,16 +342,16 @@ Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND i Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(57) CometColumnarToRow +(57) CometNativeColumnarToRow Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(58) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 11] Left keys [2]: [wr_item_sk#94, wr_order_number#95] Right keys [2]: [ws_item_sk#99, ws_order_number#101] Join type: Inner Join condition: None -(59) Project [codegen id : 15] +(59) Project [codegen id : 11] Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] @@ -360,13 +360,13 @@ Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, (61) ReusedExchange [Reuses operator id: 86] Output [1]: [d_date_sk#109] -(62) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 12] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(63) Project [codegen id : 18] +(63) Project [codegen id : 12] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -385,24 +385,24 @@ Condition : isnotnull(web_site_sk#110) Input [2]: [web_site_sk#110, web_site_id#111] Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#111, 16, true, false, true) AS web_site_id#112] -(67) CometColumnarToRow [codegen id : 17] +(67) CometNativeColumnarToRow Input [2]: [web_site_sk#110, web_site_id#112] (68) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(69) BroadcastHashJoin [codegen id : 18] +(69) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(70) Project [codegen id : 18] +(70) Project [codegen id : 12] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] -(71) HashAggregate [codegen id : 18] +(71) HashAggregate [codegen id : 12] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Keys [1]: [web_site_id#112] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] @@ -413,10 +413,10 @@ Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -(74) HashAggregate [codegen id : 19] +(74) HashAggregate [codegen id : 13] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Keys [1]: [web_site_id#112] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] @@ -425,11 +425,11 @@ Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales# (75) Union -(76) Expand [codegen id : 20] +(76) Expand [codegen id : 14] Input [5]: [sales#38, returns#39, profit#40, channel#41, id#42] Arguments: [[sales#38, returns#39, profit#40, channel#41, id#42, 0], [sales#38, returns#39, profit#40, channel#41, null, 1], [sales#38, returns#39, profit#40, null, null, 3]], [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] -(77) HashAggregate [codegen id : 20] +(77) HashAggregate [codegen id : 14] Input [6]: [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] Keys [3]: [channel#130, id#131, spark_grouping_id#132] Functions [3]: [partial_sum(sales#38), partial_sum(returns#39), partial_sum(profit#40)] @@ -440,10 +440,10 @@ Results [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 21] +(79) CometNativeColumnarToRow Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -(80) HashAggregate [codegen id : 21] +(80) HashAggregate [codegen id : 15] Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Keys [3]: [channel#130, id#131, spark_grouping_id#132] Functions [3]: [sum(sales#38), sum(returns#39), sum(profit#40)] @@ -458,7 +458,7 @@ Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel# Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (86) -+- * CometColumnarToRow (85) ++- CometNativeColumnarToRow (85) +- CometProject (84) +- CometFilter (83) +- CometNativeScan parquet spark_catalog.default.date_dim (82) @@ -479,7 +479,7 @@ Condition : (((isnotnull(d_date#151) AND (d_date#151 >= 2000-08-23)) AND (d_date Input [2]: [d_date_sk#22, d_date#151] Arguments: [d_date_sk#22], [d_date_sk#22] -(85) CometColumnarToRow [codegen id : 1] +(85) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (86) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt index 60352ef90a..15ad014521 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -30,17 +30,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -59,17 +59,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.catalog_page +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -88,17 +88,17 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/simplified.txt index 9dc4fc19c7..d84ba471ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (21) + WholeStageCodegen (15) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (20) + WholeStageCodegen (14) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_id] #2 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] BroadcastHashJoin [store_sk,s_store_sk] @@ -30,12 +30,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] WholeStageCodegen (2) Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] Filter [sr_store_sk] @@ -47,18 +45,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cp_catalog_page_id] #5 - WholeStageCodegen (11) + WholeStageCodegen (7) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] BroadcastHashJoin [page_sk,cp_catalog_page_sk] @@ -66,14 +62,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union - WholeStageCodegen (7) + WholeStageCodegen (5) Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] Filter [cs_catalog_page_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) + WholeStageCodegen (6) Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] Filter [cr_catalog_page_sk] ColumnarToRow @@ -84,18 +80,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) + CometNativeColumnarToRow + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (13) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [web_site_id] #7 - WholeStageCodegen (18) + WholeStageCodegen (12) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] BroadcastHashJoin [wsr_web_site_sk,web_site_sk] @@ -103,25 +97,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] BroadcastHashJoin [date_sk,d_date_sk] InputAdapter Union - WholeStageCodegen (13) + WholeStageCodegen (9) Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] Filter [ws_web_site_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (15) + WholeStageCodegen (11) Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] InputAdapter BroadcastExchange #8 - WholeStageCodegen (14) + WholeStageCodegen (10) ColumnarToRow InputAdapter Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] @@ -129,9 +123,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometNativeColumnarToRow + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt index b4a1dedd61..d86780b8ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt @@ -411,7 +411,7 @@ Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -432,7 +432,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt index d128acaf06..76c166e7de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt index 6fff89a25c..aa3a803ef1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index b4a1dedd61..d86780b8ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -411,7 +411,7 @@ Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -432,7 +432,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/extended.txt index d128acaf06..76c166e7de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6fff89a25c..aa3a803ef1 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/explain.txt index de9a1cd3ec..188ce4e5af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (30) +- * HashAggregate (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometColumnarExchange (27) +- * HashAggregate (26) +- * Project (25) @@ -12,7 +12,7 @@ TakeOrderedAndProject (30) : : +- * BroadcastHashJoin Inner BuildRight (15) : : :- * Project (9) : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * CometColumnarToRow (3) + : : : :- CometNativeColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (7) @@ -20,12 +20,12 @@ TakeOrderedAndProject (30) : : : +- * ColumnarToRow (5) : : : +- Scan parquet spark_catalog.default.store_returns (4) : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) + : : +- CometNativeColumnarToRow (13) : : +- CometProject (12) : : +- CometFilter (11) : : +- CometNativeScan parquet spark_catalog.default.store (10) : +- BroadcastExchange (20) - : +- * CometColumnarToRow (19) + : +- CometNativeColumnarToRow (19) : +- CometFilter (18) : +- CometNativeScan parquet spark_catalog.default.date_dim (17) +- ReusedExchange (23) @@ -43,7 +43,7 @@ ReadSchema: struct Input [1]: [d_date_sk#26] Condition : isnotnull(d_date_sk#26) -(19) CometColumnarToRow [codegen id : 3] +(19) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (20) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(21) BroadcastHashJoin [codegen id : 5] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(22) Project [codegen id : 5] +(22) Project [codegen id : 2] Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] (23) ReusedExchange [Reuses operator id: 35] Output [1]: [d_date_sk#27] -(24) BroadcastHashJoin [codegen id : 5] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#9] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(25) Project [codegen id : 5] +(25) Project [codegen id : 2] Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 2] Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] @@ -159,10 +159,10 @@ Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_na Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] -(29) HashAggregate [codegen id : 6] +(29) HashAggregate [codegen id : 3] Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] @@ -177,7 +177,7 @@ Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRS Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -198,7 +198,7 @@ Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2 Input [3]: [d_date_sk#27, d_year#48, d_moy#49] Arguments: [d_date_sk#27], [d_date_sk#27] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#27] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt index 6ece21441c..ab41730455 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- BroadcastExchange @@ -20,21 +20,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/simplified.txt index 186c041b09..fddccaf3b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -14,8 +14,8 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s BroadcastHashJoin [ss_store_sk,s_store_sk] Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] InputAdapter @@ -27,26 +27,20 @@ TakeOrderedAndProject [s_store_name,s_company_id,s_street_number,s_street_name,s Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometNativeColumnarToRow + CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk] + CometNativeColumnarToRow + CometFilter [d_date_sk] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk] InputAdapter ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/explain.txt index ff37c8f5fb..0ed16904e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/explain.txt @@ -172,7 +172,7 @@ Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -193,7 +193,7 @@ Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2 Input [3]: [d_date_sk#27, d_year#28, d_moy#29] Arguments: [d_date_sk#27], [d_date_sk#27] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#27] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt index 0fbba7d1e8..958b844c9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/simplified.txt index 98a44eaf40..fd3cd6eaa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt index ff37c8f5fb..0ed16904e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt @@ -172,7 +172,7 @@ Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -193,7 +193,7 @@ Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2 Input [3]: [d_date_sk#27, d_year#28, d_moy#29] Arguments: [d_date_sk#27], [d_date_sk#27] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [1]: [d_date_sk#27] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/extended.txt index 0fbba7d1e8..958b844c9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 98a44eaf40..fd3cd6eaa5 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #4 CometProject [s_street_type,s_suite_number,s_state,s_zip] [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] CometFilter [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/explain.txt index 9af40029c8..508b20f435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (42) +- * Filter (41) +- Window (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometSort (38) +- CometExchange (37) +- CometProject (36) @@ -11,11 +11,11 @@ TakeOrderedAndProject (42) : +- CometColumnarExchange (16) : +- * Project (15) : +- Window (14) - : +- * CometColumnarToRow (13) + : +- CometNativeColumnarToRow (13) : +- CometSort (12) : +- CometColumnarExchange (11) : +- * HashAggregate (10) - : +- * CometColumnarToRow (9) + : +- CometNativeColumnarToRow (9) : +- CometColumnarExchange (8) : +- * HashAggregate (7) : +- * Project (6) @@ -28,11 +28,11 @@ TakeOrderedAndProject (42) +- CometColumnarExchange (33) +- * Project (32) +- Window (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometSort (29) +- CometColumnarExchange (28) +- * HashAggregate (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometColumnarExchange (25) +- * HashAggregate (24) +- * Project (23) @@ -51,27 +51,27 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 47] Output [2]: [d_date_sk#5, d_date#6] -(5) BroadcastHashJoin [codegen id : 2] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 2] +(6) Project [codegen id : 1] Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -(7) HashAggregate [codegen id : 2] +(7) HashAggregate [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] @@ -82,10 +82,10 @@ Results [3]: [ws_item_sk#1, d_date#6, sum#8] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 3] +(9) CometNativeColumnarToRow Input [3]: [ws_item_sk#1, d_date#6, sum#8] -(10) HashAggregate [codegen id : 3] +(10) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] @@ -100,14 +100,14 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 4] +(13) CometNativeColumnarToRow Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] (14) Window Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 5] +(15) Project [codegen id : 3] Output [3]: [item_sk#10, d_date#6, cume_sales#12] Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] @@ -127,27 +127,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 4] Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -(20) Filter [codegen id : 7] +(20) Filter [codegen id : 4] Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) (21) ReusedExchange [Reuses operator id: 47] Output [2]: [d_date_sk#16, d_date#17] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#15] Right keys [1]: [d_date_sk#16] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 4] Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16, d_date#17] -(24) HashAggregate [codegen id : 7] +(24) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] Keys [2]: [ss_item_sk#13, d_date#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] @@ -158,10 +158,10 @@ Results [3]: [ss_item_sk#13, d_date#17, sum#19] Input [3]: [ss_item_sk#13, d_date#17, sum#19] Arguments: hashpartitioning(ss_item_sk#13, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometColumnarToRow [codegen id : 8] +(26) CometNativeColumnarToRow Input [3]: [ss_item_sk#13, d_date#17, sum#19] -(27) HashAggregate [codegen id : 8] +(27) HashAggregate [codegen id : 5] Input [3]: [ss_item_sk#13, d_date#17, sum#19] Keys [2]: [ss_item_sk#13, d_date#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] @@ -176,14 +176,14 @@ Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumna Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] Arguments: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(30) CometColumnarToRow [codegen id : 9] +(30) CometNativeColumnarToRow Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] (31) Window Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#17 ASC NULLS FIRST] -(32) Project [codegen id : 10] +(32) Project [codegen id : 6] Output [3]: [item_sk#21, d_date#17, cume_sales#23] Input [5]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13, cume_sales#23] @@ -212,14 +212,14 @@ Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] -(39) CometColumnarToRow [codegen id : 11] +(39) CometNativeColumnarToRow Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] (40) Window Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(41) Filter [codegen id : 12] +(41) Filter [codegen id : 7] Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) @@ -231,7 +231,7 @@ Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (47) -+- * CometColumnarToRow (46) ++- CometNativeColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometNativeScan parquet spark_catalog.default.date_dim (43) @@ -252,7 +252,7 @@ Condition : (((isnotnull(d_month_seq#30) AND (d_month_seq#30 >= 1200)) AND (d_mo Input [3]: [d_date_sk#5, d_date#6, d_month_seq#30] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(46) CometColumnarToRow [codegen id : 1] +(46) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (47) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index 9dc007f5e9..db4e38a1bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -10,11 +10,11 @@ TakeOrderedAndProject : +- CometColumnarExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -24,12 +24,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,11 +37,11 @@ TakeOrderedAndProject +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -51,7 +51,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/simplified.txt index 51d3f11ae6..1029c768d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/simplified.txt @@ -1,73 +1,65 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (12) + WholeStageCodegen (7) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #6 - WholeStageCodegen (10) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #8 - WholeStageCodegen (7) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 + CometNativeColumnarToRow + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (3) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk,d_date] #4 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #6 + WholeStageCodegen (6) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #7 + WholeStageCodegen (5) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk,d_date] #8 + WholeStageCodegen (4) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt index 683cc25d24..716f297aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt @@ -224,7 +224,7 @@ Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -245,7 +245,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt index 66c5717cc7..8a2badfaff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt index a5af2e114b..ba02a3e822 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt @@ -30,12 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index 683cc25d24..716f297aaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -224,7 +224,7 @@ Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) @@ -245,7 +245,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt index 66c5717cc7..8a2badfaff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a5af2e114b..ba02a3e822 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 @@ -30,12 +30,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #6 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/explain.txt index 4af1b4f562..3a3764351c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/simplified.txt index cee223d027..f5de3784b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_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 [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] - CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [d_year,brand_id,brand,ext_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 [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] + CometBroadcastHashJoin [d_date_sk,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk,d_year] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/explain.txt index f8c8dd3bf2..bf21baa1b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (30) +- * Project (29) +- * Filter (28) +- Window (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -16,7 +16,7 @@ TakeOrderedAndProject (30) : +- * BroadcastHashJoin Inner BuildRight (12) : :- * Project (10) : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject (30) : : +- Scan parquet spark_catalog.default.store_sales (5) : +- ReusedExchange (11) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -46,7 +46,7 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [2]: [i_item_sk#1, i_manufact_id#5] (5) Scan parquet spark_catalog.default.store_sales @@ -68,26 +68,26 @@ Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] (11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#15, d_qoy#16] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#13] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16] Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#16] @@ -102,24 +102,24 @@ ReadSchema: struct Input [1]: [s_store_sk#17] Condition : isnotnull(s_store_sk#17) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [s_store_sk#17] (17) BroadcastExchange Input [1]: [s_store_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#11] Right keys [1]: [s_store_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16, s_store_sk#17] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] Keys [2]: [i_manufact_id#5, d_qoy#16] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] @@ -130,10 +130,10 @@ Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] Keys [2]: [i_manufact_id#5, d_qoy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] @@ -148,18 +148,18 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColum Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] (27) Window Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 7] +(29) Project [codegen id : 4] Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] @@ -171,7 +171,7 @@ Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -192,7 +192,7 @@ Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_qoy#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt index 0b6c2edaa0..5c7a26057d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject +- Project +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -25,17 +25,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/simplified.txt index 63c3e1a17a..d427ecd731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/simplified.txt @@ -1,53 +1,47 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id,d_qoy] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometNativeColumnarToRow + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + WholeStageCodegen (3) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_manufact_id,d_qoy] #2 + WholeStageCodegen (2) + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 + CometNativeColumnarToRow + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + ReusedExchange [d_date_sk,d_qoy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt index 2fc38bf838..74b64a3eba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt index 0c123eb728..e0771ad655 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 2fc38bf838..74b64a3eba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0c123eb728..e0771ad655 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] CometBroadcastExchange [d_date_sk,d_qoy] #5 CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/explain.txt index 43120fc051..fbc2684125 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/explain.txt @@ -1,11 +1,11 @@ == Physical Plan == TakeOrderedAndProject (61) +- * HashAggregate (60) - +- * CometColumnarToRow (59) + +- CometNativeColumnarToRow (59) +- CometColumnarExchange (58) +- * HashAggregate (57) +- * HashAggregate (56) - +- * CometColumnarToRow (55) + +- CometNativeColumnarToRow (55) +- CometColumnarExchange (54) +- * HashAggregate (53) +- * Project (52) @@ -16,7 +16,7 @@ TakeOrderedAndProject (61) : : +- * BroadcastHashJoin Inner BuildRight (41) : : :- * Project (35) : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * CometColumnarToRow (29) + : : : :- CometNativeColumnarToRow (29) : : : : +- CometHashAggregate (28) : : : : +- CometColumnarExchange (27) : : : : +- * HashAggregate (26) @@ -36,13 +36,13 @@ TakeOrderedAndProject (61) : : : : : : : +- * ColumnarToRow (6) : : : : : : : +- Scan parquet spark_catalog.default.web_sales (5) : : : : : : +- BroadcastExchange (14) - : : : : : : +- * CometColumnarToRow (13) + : : : : : : +- CometNativeColumnarToRow (13) : : : : : : +- CometProject (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- ReusedExchange (17) : : : : +- BroadcastExchange (23) - : : : : +- * CometColumnarToRow (22) + : : : : +- CometNativeColumnarToRow (22) : : : : +- CometFilter (21) : : : : +- CometNativeScan parquet spark_catalog.default.customer (20) : : : +- BroadcastExchange (33) @@ -50,12 +50,12 @@ TakeOrderedAndProject (61) : : : +- * ColumnarToRow (31) : : : +- Scan parquet spark_catalog.default.store_sales (30) : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) + : : +- CometNativeColumnarToRow (39) : : +- CometProject (38) : : +- CometFilter (37) : : +- CometNativeScan parquet spark_catalog.default.customer_address (36) : +- BroadcastExchange (47) - : +- * CometColumnarToRow (46) + : +- CometNativeColumnarToRow (46) : +- CometProject (45) : +- CometFilter (44) : +- CometNativeScan parquet spark_catalog.default.store (43) @@ -117,33 +117,33 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [3]: [i_item_sk#14, i_class#15, i_category#16] Arguments: [i_item_sk#14], [i_item_sk#14] -(13) CometColumnarToRow [codegen id : 3] +(13) CometNativeColumnarToRow Input [1]: [i_item_sk#14] (14) BroadcastExchange Input [1]: [i_item_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(15) BroadcastHashJoin [codegen id : 6] +(15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(16) Project [codegen id : 6] +(16) Project [codegen id : 3] Output [2]: [sold_date_sk#5, customer_sk#6] Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] (17) ReusedExchange [Reuses operator id: 66] Output [1]: [d_date_sk#17] -(18) BroadcastHashJoin [codegen id : 6] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [sold_date_sk#5] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 6] +(19) Project [codegen id : 3] Output [1]: [customer_sk#6] Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] @@ -158,24 +158,24 @@ ReadSchema: struct Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [2]: [c_customer_sk#18, c_current_addr_sk#19] (23) BroadcastExchange Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 3] Left keys [1]: [customer_sk#6] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 3] Output [2]: [c_customer_sk#18, c_current_addr_sk#19] Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 3] Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] Functions: [] @@ -191,7 +191,7 @@ Input [2]: [c_customer_sk#18, c_current_addr_sk#19] Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] Functions: [] -(29) CometColumnarToRow [codegen id : 11] +(29) CometNativeColumnarToRow Input [2]: [c_customer_sk#18, c_current_addr_sk#19] (30) Scan parquet spark_catalog.default.store_sales @@ -202,10 +202,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] +(31) ColumnarToRow [codegen id : 4] Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -(32) Filter [codegen id : 7] +(32) Filter [codegen id : 4] Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) @@ -213,13 +213,13 @@ Condition : isnotnull(ss_customer_sk#20) Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_customer_sk#18] Right keys [1]: [ss_customer_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 11] +(35) Project [codegen id : 5] Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] @@ -238,20 +238,20 @@ Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnot Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] Arguments: [ca_address_sk#24, ca_county#25, ca_state#27], [ca_address_sk#24, ca_county#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#26, 2, true, false, true) AS ca_state#27] -(39) CometColumnarToRow [codegen id : 8] +(39) CometNativeColumnarToRow Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] (40) BroadcastExchange Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(41) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#19] Right keys [1]: [ca_address_sk#24] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(42) Project [codegen id : 5] Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27] Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#27] @@ -270,37 +270,37 @@ Condition : (isnotnull(s_county#28) AND isnotnull(staticinvoke(class org.apache. Input [2]: [s_county#28, s_state#29] Arguments: [s_county#28, s_state#30], [s_county#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#29, 2, true, false, true) AS s_state#30] -(46) CometColumnarToRow [codegen id : 9] +(46) CometNativeColumnarToRow Input [2]: [s_county#28, s_state#30] (47) BroadcastExchange Input [2]: [s_county#28, s_state#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 11] +(48) BroadcastHashJoin [codegen id : 5] Left keys [2]: [ca_county#25, ca_state#27] Right keys [2]: [s_county#28, s_state#30] Join type: Inner Join condition: None -(49) Project [codegen id : 11] +(49) Project [codegen id : 5] Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27, s_county#28, s_state#30] (50) ReusedExchange [Reuses operator id: 71] Output [1]: [d_date_sk#31] -(51) BroadcastHashJoin [codegen id : 11] +(51) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(52) Project [codegen id : 11] +(52) Project [codegen id : 5] Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#31] -(53) HashAggregate [codegen id : 11] +(53) HashAggregate [codegen id : 5] Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] Keys [1]: [c_customer_sk#18] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] @@ -311,17 +311,17 @@ Results [2]: [c_customer_sk#18, sum#33] Input [2]: [c_customer_sk#18, sum#33] Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(55) CometColumnarToRow [codegen id : 12] +(55) CometNativeColumnarToRow Input [2]: [c_customer_sk#18, sum#33] -(56) HashAggregate [codegen id : 12] +(56) HashAggregate [codegen id : 6] Input [2]: [c_customer_sk#18, sum#33] Keys [1]: [c_customer_sk#18] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#34] Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#34,17,2) / 50) as int) AS segment#35] -(57) HashAggregate [codegen id : 12] +(57) HashAggregate [codegen id : 6] Input [1]: [segment#35] Keys [1]: [segment#35] Functions [1]: [partial_count(1)] @@ -332,10 +332,10 @@ Results [2]: [segment#35, count#37] Input [2]: [segment#35, count#37] Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(59) CometColumnarToRow [codegen id : 13] +(59) CometNativeColumnarToRow Input [2]: [segment#35, count#37] -(60) HashAggregate [codegen id : 13] +(60) HashAggregate [codegen id : 7] Input [2]: [segment#35, count#37] Keys [1]: [segment#35] Functions [1]: [count(1)] @@ -350,7 +350,7 @@ Arguments: 100, [segment#35 ASC NULLS FIRST, num_customers#39 ASC NULLS FIRST], Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (66) -+- * CometColumnarToRow (65) ++- CometNativeColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometNativeScan parquet spark_catalog.default.date_dim (62) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 12 Input [3]: [d_date_sk#17, d_year#41, d_moy#42] Arguments: [d_date_sk#17], [d_date_sk#17] -(65) CometColumnarToRow [codegen id : 1] +(65) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (66) BroadcastExchange @@ -382,7 +382,7 @@ Subquery:2 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometNativeScan parquet spark_catalog.default.date_dim (67) @@ -403,7 +403,7 @@ Condition : (((isnotnull(d_month_seq#43) AND (d_month_seq#43 >= Subquery scalar- Input [2]: [d_date_sk#31, d_month_seq#43] Arguments: [d_date_sk#31], [d_date_sk#31] -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [1]: [d_date_sk#31] (71) BroadcastExchange @@ -411,7 +411,7 @@ Input [1]: [d_date_sk#31] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] Subquery:4 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* CometColumnarToRow (78) +CometNativeColumnarToRow (78) +- CometHashAggregate (77) +- CometExchange (76) +- CometHashAggregate (75) @@ -449,11 +449,11 @@ Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [1]: [(d_month_seq + 1)#49] Subquery:5 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* CometColumnarToRow (85) +CometNativeColumnarToRow (85) +- CometHashAggregate (84) +- CometExchange (83) +- CometHashAggregate (82) @@ -491,7 +491,7 @@ Input [1]: [(d_month_seq + 3)#50] Keys [1]: [(d_month_seq + 3)#50] Functions: [] -(85) CometColumnarToRow [codegen id : 1] +(85) CometNativeColumnarToRow Input [1]: [(d_month_seq + 3)#50] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt index 6ba04a3153..273efea475 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,17 +42,17 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange @@ -61,11 +61,11 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : :- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometExchange : : : : : +- CometHashAggregate @@ -73,7 +73,7 @@ TakeOrderedAndProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -82,21 +82,21 @@ TakeOrderedAndProject : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter : :- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -104,7 +104,7 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/simplified.txt index ba04de874d..3e9405f691 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/simplified.txt @@ -1,16 +1,16 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [segment] #1 - WholeStageCodegen (12) + WholeStageCodegen (6) HashAggregate [segment] [count,count] HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_sk] #2 - WholeStageCodegen (11) + WholeStageCodegen (5) HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] Project [c_customer_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -20,11 +20,11 @@ TakeOrderedAndProject [segment,num_customers,segment_base] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [c_customer_sk,c_current_addr_sk] CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [c_customer_sk,c_current_addr_sk] Project [c_customer_sk,c_current_addr_sk] BroadcastHashJoin [customer_sk,c_customer_sk] @@ -42,12 +42,10 @@ TakeOrderedAndProject [segment,num_customers,segment_base] Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] WholeStageCodegen (2) Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] Filter [ws_item_sk,ws_bill_customer_sk] @@ -57,71 +55,57 @@ TakeOrderedAndProject [segment,num_customers,segment_base] ReusedSubquery [d_date_sk] #1 InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #7 - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #9 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #10 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #3 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #9 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #10 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter BroadcastExchange #11 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_county,ca_state] + CometFilter [ca_address_sk,ca_county,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] InputAdapter BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_county,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_county,s_state] + CometFilter [s_county,s_state] + CometNativeScan parquet spark_catalog.default.store [s_county,s_state] InputAdapter ReusedExchange [d_date_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/explain.txt index 247d1d2d5a..5710e835f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/explain.txt @@ -337,7 +337,7 @@ Input [3]: [segment#40, num_customers#42, segment_base#43] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) @@ -358,7 +358,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (63) BroadcastExchange @@ -369,7 +369,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (68) -+- * CometColumnarToRow (67) ++- CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) @@ -390,7 +390,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= Subquery scalar- Input [2]: [d_date_sk#33, d_month_seq#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (68) BroadcastExchange @@ -398,7 +398,7 @@ Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:4 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* CometColumnarToRow (75) +CometNativeColumnarToRow (75) +- CometHashAggregate (74) +- CometExchange (73) +- CometHashAggregate (72) @@ -436,11 +436,11 @@ Input [1]: [(d_month_seq + 1)#45] Keys [1]: [(d_month_seq + 1)#45] Functions: [] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [(d_month_seq + 1)#45] Subquery:5 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* CometColumnarToRow (82) +CometNativeColumnarToRow (82) +- CometHashAggregate (81) +- CometExchange (80) +- CometHashAggregate (79) @@ -478,7 +478,7 @@ Input [1]: [(d_month_seq + 3)#46] Keys [1]: [(d_month_seq + 3)#46] Functions: [] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [(d_month_seq + 3)#46] Subquery:6 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt index 98fce92fb1..0b482769c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,11 +53,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : :- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometExchange : : : : : +- CometHashAggregate @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/simplified.txt index ae0cecb1ba..89c615df18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] @@ -57,32 +55,26 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #3 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 CometProject [ca_state] [ca_address_sk,ca_county,ca_state] CometFilter [ca_address_sk,ca_county,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 247d1d2d5a..5710e835f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -337,7 +337,7 @@ Input [3]: [segment#40, num_customers#42, segment_base#43] Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometFilter (60) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) @@ -358,7 +358,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#17] (63) BroadcastExchange @@ -369,7 +369,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (68) -+- * CometColumnarToRow (67) ++- CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) @@ -390,7 +390,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= Subquery scalar- Input [2]: [d_date_sk#33, d_month_seq#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (68) BroadcastExchange @@ -398,7 +398,7 @@ Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:4 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* CometColumnarToRow (75) +CometNativeColumnarToRow (75) +- CometHashAggregate (74) +- CometExchange (73) +- CometHashAggregate (72) @@ -436,11 +436,11 @@ Input [1]: [(d_month_seq + 1)#45] Keys [1]: [(d_month_seq + 1)#45] Functions: [] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [(d_month_seq + 1)#45] Subquery:5 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* CometColumnarToRow (82) +CometNativeColumnarToRow (82) +- CometHashAggregate (81) +- CometExchange (80) +- CometHashAggregate (79) @@ -478,7 +478,7 @@ Input [1]: [(d_month_seq + 3)#46] Keys [1]: [(d_month_seq + 3)#46] Functions: [] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [(d_month_seq + 3)#46] Subquery:6 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/extended.txt index 98fce92fb1..0b482769c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,11 +53,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : :- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometExchange : : : : : +- CometHashAggregate @@ -65,7 +65,7 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate 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 ae0cecb1ba..89c615df18 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 @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] @@ -57,32 +55,26 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #3 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + CometNativeColumnarToRow + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 CometProject [ca_state] [ca_address_sk,ca_county,ca_state] CometFilter [ca_address_sk,ca_county,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/explain.txt index 358eb94c69..23cbbb525e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (19) +CometNativeColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] Input [3]: [brand_id#13, brand#14, ext_price#15] Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] -(19) CometColumnarToRow [codegen id : 1] +(19) CometNativeColumnarToRow Input [3]: [brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/extended.txt index e09de16f3a..b65033bdd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/simplified.txt index a2adacc531..c31575cc36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/simplified.txt @@ -1,21 +1,19 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_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 [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] - CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] +CometNativeColumnarToRow + CometTakeOrderedAndProject [brand_id,brand,ext_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 [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] + CometBroadcastHashJoin [d_date_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] #2 + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/explain.txt index 62396854a7..332b99e63f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (67) +- * HashAggregate (66) - +- * CometColumnarToRow (65) + +- CometNativeColumnarToRow (65) +- CometColumnarExchange (64) +- * HashAggregate (63) +- Union (62) :- * HashAggregate (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometColumnarExchange (27) : +- * HashAggregate (26) : +- * Project (25) @@ -20,12 +20,12 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometProject (21) : +- CometBroadcastHashJoin (20) : :- CometFilter (15) @@ -35,7 +35,7 @@ TakeOrderedAndProject (67) : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.item (16) :- * HashAggregate (45) - : +- * CometColumnarToRow (44) + : +- CometNativeColumnarToRow (44) : +- CometColumnarExchange (43) : +- * HashAggregate (42) : +- * Project (41) @@ -51,7 +51,7 @@ TakeOrderedAndProject (67) : : +- ReusedExchange (36) : +- ReusedExchange (39) +- * HashAggregate (61) - +- * CometColumnarToRow (60) + +- CometNativeColumnarToRow (60) +- CometColumnarExchange (59) +- * HashAggregate (58) +- * Project (57) @@ -76,23 +76,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] @@ -111,20 +111,20 @@ Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isno Input [2]: [ca_address_sk#7, ca_gmt_offset#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [ca_address_sk#7] (11) BroadcastExchange Input [1]: [ca_address_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] @@ -166,24 +166,24 @@ Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCod Input [2]: [i_item_sk#9, i_item_id#10] Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true) AS i_item_id#14] -(22) CometColumnarToRow [codegen id : 3] +(22) CometNativeColumnarToRow Input [2]: [i_item_sk#9, i_item_id#14] (23) BroadcastExchange Input [2]: [i_item_sk#9, i_item_id#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 4] +(24) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#9] Join type: Inner Join condition: None -(25) Project [codegen id : 4] +(25) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#3, i_item_id#14] Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] -(26) HashAggregate [codegen id : 4] +(26) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#14] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -194,10 +194,10 @@ Results [2]: [i_item_id#14, sum#16] Input [2]: [i_item_id#14, sum#16] Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [2]: [i_item_id#14, sum#16] -(29) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 2] Input [2]: [i_item_id#14, sum#16] Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -212,53 +212,53 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 9] +(31) ColumnarToRow [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -(32) Filter [codegen id : 9] +(32) Filter [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) (33) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#24] -(34) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#23] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(35) Project [codegen id : 9] +(35) Project [codegen id : 3] Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] (36) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#25] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_addr_sk#20] Right keys [1]: [ca_address_sk#25] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 3] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] (39) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#26, i_item_id#27] -(40) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#21] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(41) Project [codegen id : 3] Output [2]: [cs_ext_sales_price#22, i_item_id#27] Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 3] Input [2]: [cs_ext_sales_price#22, i_item_id#27] Keys [1]: [i_item_id#27] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] @@ -269,10 +269,10 @@ Results [2]: [i_item_id#27, sum#29] Input [2]: [i_item_id#27, sum#29] Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometColumnarToRow [codegen id : 10] +(44) CometNativeColumnarToRow Input [2]: [i_item_id#27, sum#29] -(45) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 4] Input [2]: [i_item_id#27, sum#29] Keys [1]: [i_item_id#27] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] @@ -287,53 +287,53 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 14] +(47) ColumnarToRow [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -(48) Filter [codegen id : 14] +(48) Filter [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) (49) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#36] -(50) BroadcastHashJoin [codegen id : 14] +(50) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#35] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(51) Project [codegen id : 14] +(51) Project [codegen id : 5] Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] (52) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#37] -(53) BroadcastHashJoin [codegen id : 14] +(53) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_bill_addr_sk#33] Right keys [1]: [ca_address_sk#37] Join type: Inner Join condition: None -(54) Project [codegen id : 14] +(54) Project [codegen id : 5] Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] (55) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#38, i_item_id#39] -(56) BroadcastHashJoin [codegen id : 14] +(56) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_item_sk#32] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(57) Project [codegen id : 5] Output [2]: [ws_ext_sales_price#34, i_item_id#39] Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] -(58) HashAggregate [codegen id : 14] +(58) HashAggregate [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#39] Keys [1]: [i_item_id#39] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] @@ -344,10 +344,10 @@ Results [2]: [i_item_id#39, sum#41] Input [2]: [i_item_id#39, sum#41] Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(60) CometColumnarToRow [codegen id : 15] +(60) CometNativeColumnarToRow Input [2]: [i_item_id#39, sum#41] -(61) HashAggregate [codegen id : 15] +(61) HashAggregate [codegen id : 6] Input [2]: [i_item_id#39, sum#41] Keys [1]: [i_item_id#39] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] @@ -356,7 +356,7 @@ Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34) (62) Union -(63) HashAggregate [codegen id : 16] +(63) HashAggregate [codegen id : 7] Input [2]: [i_item_id#18, total_sales#19] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(total_sales#19)] @@ -367,10 +367,10 @@ Results [3]: [i_item_id#18, sum#46, isEmpty#47] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(65) CometColumnarToRow [codegen id : 17] +(65) CometNativeColumnarToRow Input [3]: [i_item_id#18, sum#46, isEmpty#47] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 8] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Keys [1]: [i_item_id#18] Functions [1]: [sum(total_sales#19)] @@ -385,7 +385,7 @@ Arguments: 100, [total_sales#49 ASC NULLS FIRST], [i_item_id#18, total_sales#49] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometNativeScan parquet spark_catalog.default.date_dim (68) @@ -406,7 +406,7 @@ Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2 Input [3]: [d_date_sk#6, d_year#50, d_moy#51] Arguments: [d_date_sk#6], [d_date_sk#6] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (72) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt index 7a2832757d..ad055b48b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,22 +19,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -44,7 +44,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,17 +58,17 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -78,7 +78,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -92,17 +92,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/simplified.txt index 81d90bdfb2..4d9a45fc72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -27,41 +27,35 @@ TakeOrderedAndProject [total_sales,i_item_id] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (10) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #6 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_color] + CometNativeScan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -80,12 +74,12 @@ TakeOrderedAndProject [total_sales,i_item_id] ReusedExchange [ca_address_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] BroadcastHashJoin [ws_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/explain.txt index 5562e86b91..7f18d8071c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/simplified.txt index 19eca782cd..eb64094749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 5562e86b91..7f18d8071c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 19eca782cd..eb64094749 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/explain.txt index c3ba92766c..b63efdd787 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.call_center (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.catalog_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [2]: [cc_call_center_sk#14, cc_name#15] (17) BroadcastExchange Input [2]: [cc_call_center_sk#14, cc_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_call_center_sk#6] Right keys [1]: [cc_call_center_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] @@ -151,10 +151,10 @@ Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] (27) Window Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] (32) ReusedExchange [Reuses operator id: 21] Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQU Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] (38) Window Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] @@ -228,13 +228,13 @@ Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales# Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] @@ -245,14 +245,14 @@ Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] (46) Window Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] @@ -260,13 +260,13 @@ Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales# Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt index 65c66a7da8..66c510aaa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/simplified.txt index 4f9ac35f4d..c085ecfb51 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [cs_item_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [cc_call_center_sk,cc_name] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt index eb21a3abd7..24505a9158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt index b854e818db..2f01d9b13f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index eb21a3abd7..24505a9158 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 b854e818db..2f01d9b13f 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/explain.txt index 7d820f4024..5113f59847 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (53) : +- * BroadcastHashJoin Inner BuildRight (34) : :- * Filter (18) : : +- * HashAggregate (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- * HashAggregate (14) : : +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (53) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -25,7 +25,7 @@ TakeOrderedAndProject (53) : +- BroadcastExchange (33) : +- * Filter (32) : +- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * Project (27) @@ -40,7 +40,7 @@ TakeOrderedAndProject (53) +- BroadcastExchange (50) +- * Filter (49) +- * HashAggregate (48) - +- * CometColumnarToRow (47) + +- CometNativeColumnarToRow (47) +- CometColumnarExchange (46) +- * HashAggregate (45) +- * Project (44) @@ -62,10 +62,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) @@ -84,37 +84,37 @@ Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache. Input [2]: [i_item_sk#5, i_item_id#6] Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [i_item_sk#5, i_item_id#7] (8) BroadcastExchange Input [2]: [i_item_sk#5, i_item_id#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] (11) ReusedExchange [Reuses operator id: 63] Output [1]: [d_date_sk#8] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#2, i_item_id#7] Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#2, i_item_id#7] Keys [1]: [i_item_id#7] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -125,17 +125,17 @@ Results [2]: [i_item_id#7, sum#10] Input [2]: [i_item_id#7, sum#10] Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 12] +(16) CometNativeColumnarToRow Input [2]: [i_item_id#7, sum#10] -(17) HashAggregate [codegen id : 12] +(17) HashAggregate [codegen id : 6] Input [2]: [i_item_id#7, sum#10] Keys [1]: [i_item_id#7] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] Results [2]: [i_item_id#7 AS item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS ss_item_rev#13] -(18) Filter [codegen id : 12] +(18) Filter [codegen id : 6] Input [2]: [item_id#12, ss_item_rev#13] Condition : isnotnull(ss_item_rev#13) @@ -147,40 +147,40 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 2] Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -(21) Filter [codegen id : 6] +(21) Filter [codegen id : 2] Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] Condition : isnotnull(cs_item_sk#14) (22) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#18, i_item_id#19] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_item_sk#14] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(24) Project [codegen id : 2] Output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] (25) ReusedExchange [Reuses operator id: 77] Output [1]: [d_date_sk#20] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#16] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 2] Output [2]: [cs_ext_sales_price#15, i_item_id#19] Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [2]: [cs_ext_sales_price#15, i_item_id#19] Keys [1]: [i_item_id#19] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] @@ -191,17 +191,17 @@ Results [2]: [i_item_id#19, sum#22] Input [2]: [i_item_id#19, sum#22] Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [2]: [i_item_id#19, sum#22] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [2]: [i_item_id#19, sum#22] Keys [1]: [i_item_id#19] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#15))#23] Results [2]: [i_item_id#19 AS item_id#24, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#15))#23,17,2) AS cs_item_rev#25] -(32) Filter [codegen id : 7] +(32) Filter [codegen id : 3] Input [2]: [item_id#24, cs_item_rev#25] Condition : isnotnull(cs_item_rev#25) @@ -209,13 +209,13 @@ Condition : isnotnull(cs_item_rev#25) Input [2]: [item_id#24, cs_item_rev#25] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(34) BroadcastHashJoin [codegen id : 12] +(34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#24] Join type: Inner Join condition: ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#25)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#25))) AND (cast(cs_item_rev#25 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#25 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) -(35) Project [codegen id : 12] +(35) Project [codegen id : 6] Output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#25] Input [4]: [item_id#12, ss_item_rev#13, item_id#24, cs_item_rev#25] @@ -227,40 +227,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#28), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] +(37) ColumnarToRow [codegen id : 4] Input [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] -(38) Filter [codegen id : 10] +(38) Filter [codegen id : 4] Input [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] Condition : isnotnull(ws_item_sk#26) (39) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#29, i_item_id#30] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#26] Right keys [1]: [i_item_sk#29] Join type: Inner Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 4] Output [3]: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#30] Input [5]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_sk#29, i_item_id#30] (42) ReusedExchange [Reuses operator id: 77] Output [1]: [d_date_sk#31] -(43) BroadcastHashJoin [codegen id : 10] +(43) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#28] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(44) Project [codegen id : 10] +(44) Project [codegen id : 4] Output [2]: [ws_ext_sales_price#27, i_item_id#30] Input [4]: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#30, d_date_sk#31] -(45) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 4] Input [2]: [ws_ext_sales_price#27, i_item_id#30] Keys [1]: [i_item_id#30] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#27))] @@ -271,17 +271,17 @@ Results [2]: [i_item_id#30, sum#33] Input [2]: [i_item_id#30, sum#33] Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(47) CometColumnarToRow [codegen id : 11] +(47) CometNativeColumnarToRow Input [2]: [i_item_id#30, sum#33] -(48) HashAggregate [codegen id : 11] +(48) HashAggregate [codegen id : 5] Input [2]: [i_item_id#30, sum#33] Keys [1]: [i_item_id#30] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#27))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#27))#34] Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#27))#34,17,2) AS ws_item_rev#36] -(49) Filter [codegen id : 11] +(49) Filter [codegen id : 5] Input [2]: [item_id#35, ws_item_rev#36] Condition : isnotnull(ws_item_rev#36) @@ -289,13 +289,13 @@ Condition : isnotnull(ws_item_rev#36) Input [2]: [item_id#35, ws_item_rev#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(51) BroadcastHashJoin [codegen id : 12] +(51) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#35] Join type: Inner Join condition: ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#25 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#25 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#25))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#25))) -(52) Project [codegen id : 12] +(52) Project [codegen id : 6] Output [8]: [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#25, (((cs_item_rev#25 / ((ss_item_rev#13 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#13 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#13 + cs_item_rev#25) + ws_item_rev#36) / 3) AS average#40] Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#25, item_id#35, ws_item_rev#36] @@ -307,7 +307,7 @@ Arguments: 100, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (63) -+- * CometColumnarToRow (62) ++- CometNativeColumnarToRow (62) +- CometProject (61) +- CometBroadcastHashJoin (60) :- CometFilter (55) @@ -357,7 +357,7 @@ Arguments: [d_date#41], [d_date#41#45], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#41] Arguments: [d_date_sk#8], [d_date_sk#8] -(62) CometColumnarToRow [codegen id : 1] +(62) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (63) BroadcastExchange @@ -365,7 +365,7 @@ Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:2 Hosting operator id = 57 Hosting Expression = Subquery scalar-subquery#43, [id=#44] -* CometColumnarToRow (67) +CometNativeColumnarToRow (67) +- CometProject (66) +- CometFilter (65) +- CometNativeScan parquet spark_catalog.default.date_dim (64) @@ -386,12 +386,12 @@ Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) Input [2]: [d_date#41, d_week_seq#42] Arguments: [d_week_seq#42], [d_week_seq#42] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_week_seq#42] Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#17 BroadcastExchange (77) -+- * CometColumnarToRow (76) ++- CometNativeColumnarToRow (76) +- CometProject (75) +- CometBroadcastHashJoin (74) :- CometFilter (69) @@ -441,7 +441,7 @@ Arguments: [d_date#46], [d_date#41], LeftSemi, BuildRight Input [2]: [d_date_sk#20, d_date#46] Arguments: [d_date_sk#20], [d_date_sk#20] -(76) CometColumnarToRow [codegen id : 1] +(76) CometNativeColumnarToRow Input [1]: [d_date_sk#20] (77) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt index ed82c835cc..be602e0f0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -26,18 +26,18 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -54,7 +54,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -66,7 +66,7 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -77,12 +77,12 @@ TakeOrderedAndProject : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -95,7 +95,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -107,12 +107,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/simplified.txt index ab910f96e0..41a220718b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/simplified.txt @@ -1,15 +1,15 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (12) + WholeStageCodegen (6) Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] Project [item_id,ss_item_rev,cs_item_rev] BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] Filter [ss_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -21,43 +21,37 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] [d_date] + CometFilter [d_date,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (3) Filter [cs_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -69,31 +63,29 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #8 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #8 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter ReusedExchange [i_item_sk,i_item_id] #4 InputAdapter ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #9 - WholeStageCodegen (11) + WholeStageCodegen (5) Filter [ws_item_rev] HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #10 - WholeStageCodegen (10) + WholeStageCodegen (4) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/explain.txt index 85c8622ee4..b30d38143b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/explain.txt @@ -321,7 +321,7 @@ Input [8]: [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometBroadcastHashJoin (64) :- CometFilter (59) @@ -371,7 +371,7 @@ Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (67) BroadcastExchange @@ -379,7 +379,7 @@ Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* CometColumnarToRow (71) +CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -400,12 +400,12 @@ Condition : (isnotnull(d_date#9) AND (d_date#9 = 2000-01-03)) Input [2]: [d_date#9, d_week_seq#39] Arguments: [d_week_seq#39], [d_week_seq#39] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_week_seq#39] Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#17 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometProject (79) +- CometBroadcastHashJoin (78) :- CometFilter (73) @@ -455,7 +455,7 @@ Arguments: [d_date#21], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#20, d_date#21] Arguments: [d_date_sk#20], [d_date_sk#20] -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [1]: [d_date_sk#20] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt index c2dd549ac6..e503b00314 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -25,7 +25,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -43,7 +43,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -61,7 +61,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/simplified.txt index 2e709c1b54..efc1861e8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/simplified.txt @@ -18,24 +18,20 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] [d_date] + CometFilter [d_date,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] @@ -59,18 +55,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #9 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #9 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] ReusedExchange [i_item_sk,i_item_id] #4 CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index 85c8622ee4..b30d38143b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -321,7 +321,7 @@ Input [8]: [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometBroadcastHashJoin (64) :- CometFilter (59) @@ -371,7 +371,7 @@ Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (67) BroadcastExchange @@ -379,7 +379,7 @@ Input [1]: [d_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* CometColumnarToRow (71) +CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -400,12 +400,12 @@ Condition : (isnotnull(d_date#9) AND (d_date#9 = 2000-01-03)) Input [2]: [d_date#9, d_week_seq#39] Arguments: [d_week_seq#39], [d_week_seq#39] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_week_seq#39] Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#17 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometProject (79) +- CometBroadcastHashJoin (78) :- CometFilter (73) @@ -455,7 +455,7 @@ Arguments: [d_date#21], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#20, d_date#21] Arguments: [d_date_sk#20], [d_date_sk#20] -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [1]: [d_date_sk#20] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/extended.txt index c2dd549ac6..e503b00314 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -25,7 +25,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -43,7 +43,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -61,7 +61,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter 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 2e709c1b54..efc1861e8e 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 @@ -18,24 +18,20 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] [d_date] + CometFilter [d_date,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #4 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] @@ -59,18 +55,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #9 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #9 + CometProject [d_date] + CometFilter [d_date,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] ReusedExchange [i_item_sk,i_item_id] #4 CometBroadcastExchange [d_date_sk] #10 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/explain.txt index d25b1ea057..a1303cc7bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (42) +CometNativeColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometProject (40) +- CometBroadcastHashJoin (39) @@ -234,6 +234,6 @@ Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_ Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#59,(mon_sales1 / mon_sales2)#60,(tue_sales1 / tue_sales2)#61,(wed_sales1 / wed_sales2)#62,(thu_sales1 / thu_sales2)#63,(fri_sales1 / fri_sales2)#64,(sat_sales1 / sat_sales2)#65]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] -(42) CometColumnarToRow [codegen id : 1] +(42) CometNativeColumnarToRow Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/extended.txt index bfd467b72b..e23616d099 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/simplified.txt index 72823b14c1..16c54015b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/simplified.txt @@ -1,44 +1,42 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [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_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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [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] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] +CometNativeColumnarToRow + CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] + CometHashAggregate [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_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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 + CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] + CometHashAggregate [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] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [d_week_seq] #7 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/explain.txt index fb396f2dc2..530911b199 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (40) +- * Filter (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * Project (34) @@ -11,7 +11,7 @@ TakeOrderedAndProject (40) : +- * BroadcastHashJoin Inner BuildRight (17) : :- * Project (15) : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) + : : :- CometNativeColumnarToRow (9) : : : +- CometProject (8) : : : +- CometBroadcastHashJoin (7) : : : :- CometProject (3) @@ -26,7 +26,7 @@ TakeOrderedAndProject (40) : : +- Scan parquet spark_catalog.default.store_sales (10) : +- ReusedExchange (16) +- BroadcastExchange (32) - +- * CometColumnarToRow (31) + +- CometNativeColumnarToRow (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (20) @@ -80,7 +80,7 @@ Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(9) CometColumnarToRow [codegen id : 4] +(9) CometNativeColumnarToRow Input [2]: [ca_state#3, c_customer_sk#4] (10) Scan parquet spark_catalog.default.store_sales @@ -102,26 +102,26 @@ Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#4] Right keys [1]: [ss_customer_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] (16) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 2] Output [2]: [ca_state#3, ss_item_sk#6] Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] @@ -181,24 +181,24 @@ Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCod Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#17, i_category#14] Arguments: [i_item_sk#11], [i_item_sk#11] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [i_item_sk#11] (32) BroadcastExchange Input [1]: [i_item_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 4] +(33) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#6] Right keys [1]: [i_item_sk#11] Join type: Inner Join condition: None -(34) Project [codegen id : 4] +(34) Project [codegen id : 2] Output [1]: [ca_state#3] Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] -(35) HashAggregate [codegen id : 4] +(35) HashAggregate [codegen id : 2] Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] @@ -209,17 +209,17 @@ Results [2]: [ca_state#3, count#19] Input [2]: [ca_state#3, count#19] Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometColumnarToRow [codegen id : 5] +(37) CometNativeColumnarToRow Input [2]: [ca_state#3, count#19] -(38) HashAggregate [codegen id : 5] +(38) HashAggregate [codegen id : 3] Input [2]: [ca_state#3, count#19] Keys [1]: [ca_state#3] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#20] Results [2]: [ca_state#3 AS state#21, count(1)#20 AS cnt#22] -(39) Filter [codegen id : 5] +(39) Filter [codegen id : 3] Input [2]: [state#21, cnt#22] Condition : (cnt#22 >= 10) @@ -231,7 +231,7 @@ Arguments: 100, [cnt#22 ASC NULLS FIRST], [state#21, cnt#22] Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#23] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#23] Keys [1]: [d_month_seq#23] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#23] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt index fe3b28849a..f2cdb50e03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometProject @@ -25,11 +25,11 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -38,11 +38,11 @@ TakeOrderedAndProject : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -51,7 +51,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/simplified.txt index cec4eecbc6..fed4bb3845 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) + WholeStageCodegen (3) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -13,8 +13,8 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ca_state,ss_item_sk,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ca_state,c_customer_sk] CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] CometProject [ca_state] [ca_address_sk,ca_state] @@ -32,38 +32,32 @@ TakeOrderedAndProject [cnt,state] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #7 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #8 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_current_price,i_category] + CometFilter [i_current_price,i_category] + CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/explain.txt index b0905506ee..d6bc3fdb7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#22, cnt#23] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#11] Keys [1]: [d_month_seq#11] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#11] Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt index f5b69fc6cf..20115b61c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt @@ -23,11 +23,11 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/simplified.txt index 2c772b5998..2f287bd993 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/simplified.txt @@ -25,22 +25,18 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index b0905506ee..d6bc3fdb7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#22, cnt#23] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#11] Keys [1]: [d_month_seq#11] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#11] Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/extended.txt index f5b69fc6cf..20115b61c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/extended.txt @@ -23,11 +23,11 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate 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 2c772b5998..2f287bd993 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 @@ -25,22 +25,18 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/explain.txt index 5584119109..573325eb66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (67) +- * HashAggregate (66) - +- * CometColumnarToRow (65) + +- CometNativeColumnarToRow (65) +- CometColumnarExchange (64) +- * HashAggregate (63) +- Union (62) :- * HashAggregate (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometColumnarExchange (27) : +- * HashAggregate (26) : +- * Project (25) @@ -20,12 +20,12 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometProject (21) : +- CometBroadcastHashJoin (20) : :- CometFilter (15) @@ -35,7 +35,7 @@ TakeOrderedAndProject (67) : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.item (16) :- * HashAggregate (45) - : +- * CometColumnarToRow (44) + : +- CometNativeColumnarToRow (44) : +- CometColumnarExchange (43) : +- * HashAggregate (42) : +- * Project (41) @@ -51,7 +51,7 @@ TakeOrderedAndProject (67) : : +- ReusedExchange (36) : +- ReusedExchange (39) +- * HashAggregate (61) - +- * CometColumnarToRow (60) + +- CometNativeColumnarToRow (60) +- CometColumnarExchange (59) +- * HashAggregate (58) +- * Project (57) @@ -76,23 +76,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] @@ -111,20 +111,20 @@ Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isno Input [2]: [ca_address_sk#7, ca_gmt_offset#8] Arguments: [ca_address_sk#7], [ca_address_sk#7] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [ca_address_sk#7] (11) BroadcastExchange Input [1]: [ca_address_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] @@ -166,24 +166,24 @@ Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCod Input [2]: [i_item_sk#9, i_item_id#10] Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true) AS i_item_id#14] -(22) CometColumnarToRow [codegen id : 3] +(22) CometNativeColumnarToRow Input [2]: [i_item_sk#9, i_item_id#14] (23) BroadcastExchange Input [2]: [i_item_sk#9, i_item_id#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(24) BroadcastHashJoin [codegen id : 4] +(24) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#9] Join type: Inner Join condition: None -(25) Project [codegen id : 4] +(25) Project [codegen id : 1] Output [2]: [ss_ext_sales_price#3, i_item_id#14] Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] -(26) HashAggregate [codegen id : 4] +(26) HashAggregate [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#14] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -194,10 +194,10 @@ Results [2]: [i_item_id#14, sum#16] Input [2]: [i_item_id#14, sum#16] Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [2]: [i_item_id#14, sum#16] -(29) HashAggregate [codegen id : 5] +(29) HashAggregate [codegen id : 2] Input [2]: [i_item_id#14, sum#16] Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] @@ -212,53 +212,53 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 9] +(31) ColumnarToRow [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -(32) Filter [codegen id : 9] +(32) Filter [codegen id : 3] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) (33) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#24] -(34) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#23] Right keys [1]: [d_date_sk#24] Join type: Inner Join condition: None -(35) Project [codegen id : 9] +(35) Project [codegen id : 3] Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] (36) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#25] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_addr_sk#20] Right keys [1]: [ca_address_sk#25] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 3] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] (39) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#26, i_item_id#27] -(40) BroadcastHashJoin [codegen id : 9] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#21] Right keys [1]: [i_item_sk#26] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(41) Project [codegen id : 3] Output [2]: [cs_ext_sales_price#22, i_item_id#27] Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] -(42) HashAggregate [codegen id : 9] +(42) HashAggregate [codegen id : 3] Input [2]: [cs_ext_sales_price#22, i_item_id#27] Keys [1]: [i_item_id#27] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] @@ -269,10 +269,10 @@ Results [2]: [i_item_id#27, sum#29] Input [2]: [i_item_id#27, sum#29] Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(44) CometColumnarToRow [codegen id : 10] +(44) CometNativeColumnarToRow Input [2]: [i_item_id#27, sum#29] -(45) HashAggregate [codegen id : 10] +(45) HashAggregate [codegen id : 4] Input [2]: [i_item_id#27, sum#29] Keys [1]: [i_item_id#27] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] @@ -287,53 +287,53 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 14] +(47) ColumnarToRow [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -(48) Filter [codegen id : 14] +(48) Filter [codegen id : 5] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) (49) ReusedExchange [Reuses operator id: 72] Output [1]: [d_date_sk#36] -(50) BroadcastHashJoin [codegen id : 14] +(50) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#35] Right keys [1]: [d_date_sk#36] Join type: Inner Join condition: None -(51) Project [codegen id : 14] +(51) Project [codegen id : 5] Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] (52) ReusedExchange [Reuses operator id: 11] Output [1]: [ca_address_sk#37] -(53) BroadcastHashJoin [codegen id : 14] +(53) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_bill_addr_sk#33] Right keys [1]: [ca_address_sk#37] Join type: Inner Join condition: None -(54) Project [codegen id : 14] +(54) Project [codegen id : 5] Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] (55) ReusedExchange [Reuses operator id: 23] Output [2]: [i_item_sk#38, i_item_id#39] -(56) BroadcastHashJoin [codegen id : 14] +(56) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_item_sk#32] Right keys [1]: [i_item_sk#38] Join type: Inner Join condition: None -(57) Project [codegen id : 14] +(57) Project [codegen id : 5] Output [2]: [ws_ext_sales_price#34, i_item_id#39] Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] -(58) HashAggregate [codegen id : 14] +(58) HashAggregate [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#39] Keys [1]: [i_item_id#39] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] @@ -344,10 +344,10 @@ Results [2]: [i_item_id#39, sum#41] Input [2]: [i_item_id#39, sum#41] Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(60) CometColumnarToRow [codegen id : 15] +(60) CometNativeColumnarToRow Input [2]: [i_item_id#39, sum#41] -(61) HashAggregate [codegen id : 15] +(61) HashAggregate [codegen id : 6] Input [2]: [i_item_id#39, sum#41] Keys [1]: [i_item_id#39] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] @@ -356,7 +356,7 @@ Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34) (62) Union -(63) HashAggregate [codegen id : 16] +(63) HashAggregate [codegen id : 7] Input [2]: [i_item_id#18, total_sales#19] Keys [1]: [i_item_id#18] Functions [1]: [partial_sum(total_sales#19)] @@ -367,10 +367,10 @@ Results [3]: [i_item_id#18, sum#46, isEmpty#47] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(65) CometColumnarToRow [codegen id : 17] +(65) CometNativeColumnarToRow Input [3]: [i_item_id#18, sum#46, isEmpty#47] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 8] Input [3]: [i_item_id#18, sum#46, isEmpty#47] Keys [1]: [i_item_id#18] Functions [1]: [sum(total_sales#19)] @@ -385,7 +385,7 @@ Arguments: 100, [i_item_id#18 ASC NULLS FIRST, total_sales#49 ASC NULLS FIRST], Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (72) -+- * CometColumnarToRow (71) ++- CometNativeColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometNativeScan parquet spark_catalog.default.date_dim (68) @@ -406,7 +406,7 @@ Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 1 Input [3]: [d_date_sk#6, d_year#50, d_moy#51] Arguments: [d_date_sk#6], [d_date_sk#6] -(71) CometColumnarToRow [codegen id : 1] +(71) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (72) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt index 7a2832757d..ad055b48b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,22 +19,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -44,7 +44,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,17 +58,17 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -78,7 +78,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -92,17 +92,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/simplified.txt index 4c05038b34..56a5af7cf7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -27,41 +27,35 @@ TakeOrderedAndProject [i_item_id,total_sales] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (10) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometBroadcastExchange [i_item_id] #6 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (4) HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -80,12 +74,12 @@ TakeOrderedAndProject [i_item_id,total_sales] ReusedExchange [ca_address_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) + WholeStageCodegen (6) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) + WholeStageCodegen (5) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] BroadcastHashJoin [ws_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/explain.txt index e5e165b5ec..0c43a7c934 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/simplified.txt index 086da3b776..92a33e24ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index e5e165b5ec..0c43a7c934 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -350,7 +350,7 @@ Input [2]: [i_item_id#38, total_sales#44] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (67) -+- * CometColumnarToRow (66) ++- CometNativeColumnarToRow (66) +- CometProject (65) +- CometFilter (64) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) @@ -371,7 +371,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(66) CometColumnarToRow [codegen id : 1] +(66) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (67) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/extended.txt index 36b9a8306d..22c07c4964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 086da3b776..92a33e24ce 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/explain.txt index e50623a19c..34d53d4d65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/explain.txt @@ -2,7 +2,7 @@ * Project (69) +- * BroadcastNestedLoopJoin Inner BuildRight (68) :- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -21,33 +21,33 @@ : : : : : : : +- * ColumnarToRow (2) : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) + : : : : : : +- CometNativeColumnarToRow (7) : : : : : : +- CometProject (6) : : : : : : +- CometFilter (5) : : : : : : +- CometNativeScan parquet spark_catalog.default.store (4) : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) + : : : : : +- CometNativeColumnarToRow (14) : : : : : +- CometProject (13) : : : : : +- CometFilter (12) : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (11) : : : : +- ReusedExchange (18) : : : +- BroadcastExchange (24) - : : : +- * CometColumnarToRow (23) + : : : +- CometNativeColumnarToRow (23) : : : +- CometFilter (22) : : : +- CometNativeScan parquet spark_catalog.default.customer (21) : : +- BroadcastExchange (31) - : : +- * CometColumnarToRow (30) + : : +- CometNativeColumnarToRow (30) : : +- CometProject (29) : : +- CometFilter (28) : : +- CometNativeScan parquet spark_catalog.default.customer_address (27) : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometProject (36) : +- CometFilter (35) : +- CometNativeScan parquet spark_catalog.default.item (34) +- BroadcastExchange (67) +- * HashAggregate (66) - +- * CometColumnarToRow (65) + +- CometNativeColumnarToRow (65) +- CometColumnarExchange (64) +- * HashAggregate (63) +- * Project (62) @@ -78,10 +78,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 1] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) @@ -100,20 +100,20 @@ Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotn Input [2]: [s_store_sk#8, s_gmt_offset#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (8) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 1] Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] @@ -132,33 +132,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] Arguments: [p_promo_sk#10], [p_promo_sk#10] -(14) CometColumnarToRow [codegen id : 2] +(14) CometNativeColumnarToRow Input [1]: [p_promo_sk#10] (15) BroadcastExchange Input [1]: [p_promo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 7] +(16) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_promo_sk#4] Right keys [1]: [p_promo_sk#10] Join type: Inner Join condition: None -(17) Project [codegen id : 7] +(17) Project [codegen id : 1] Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] (18) ReusedExchange [Reuses operator id: 74] Output [1]: [d_date_sk#14] -(19) BroadcastHashJoin [codegen id : 7] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 7] +(20) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] @@ -173,20 +173,20 @@ ReadSchema: struct Input [2]: [c_customer_sk#15, c_current_addr_sk#16] Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [c_customer_sk#15, c_current_addr_sk#16] (24) BroadcastExchange Input [2]: [c_customer_sk#15, c_current_addr_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 7] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 7] +(26) Project [codegen id : 1] Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16] Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#15, c_current_addr_sk#16] @@ -205,20 +205,20 @@ Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -5.00)) AND is Input [2]: [ca_address_sk#17, ca_gmt_offset#18] Arguments: [ca_address_sk#17], [ca_address_sk#17] -(30) CometColumnarToRow [codegen id : 5] +(30) CometNativeColumnarToRow Input [1]: [ca_address_sk#17] (31) BroadcastExchange Input [1]: [ca_address_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#16] Right keys [1]: [ca_address_sk#17] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16, ca_address_sk#17] @@ -237,24 +237,24 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [i_item_sk#19, i_category#20] Arguments: [i_item_sk#19], [i_item_sk#19] -(37) CometColumnarToRow [codegen id : 6] +(37) CometNativeColumnarToRow Input [1]: [i_item_sk#19] (38) BroadcastExchange Input [1]: [i_item_sk#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 7] +(39) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#19] Join type: Inner Join condition: None -(40) Project [codegen id : 7] +(40) Project [codegen id : 1] Output [1]: [ss_ext_sales_price#5] Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#19] -(41) HashAggregate [codegen id : 7] +(41) HashAggregate [codegen id : 1] Input [1]: [ss_ext_sales_price#5] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] @@ -265,10 +265,10 @@ Results [1]: [sum#22] Input [1]: [sum#22] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 15] +(43) CometNativeColumnarToRow Input [1]: [sum#22] -(44) HashAggregate [codegen id : 15] +(44) HashAggregate [codegen id : 4] Input [1]: [sum#22] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] @@ -283,79 +283,79 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 13] +(46) ColumnarToRow [codegen id : 2] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -(47) Filter [codegen id : 13] +(47) Filter [codegen id : 2] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [s_store_sk#30] -(49) BroadcastHashJoin [codegen id : 13] +(49) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#27] Right keys [1]: [s_store_sk#30] Join type: Inner Join condition: None -(50) Project [codegen id : 13] +(50) Project [codegen id : 2] Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#30] (51) ReusedExchange [Reuses operator id: 74] Output [1]: [d_date_sk#31] -(52) BroadcastHashJoin [codegen id : 13] +(52) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#29] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(53) Project [codegen id : 13] +(53) Project [codegen id : 2] Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] (54) ReusedExchange [Reuses operator id: 24] Output [2]: [c_customer_sk#32, c_current_addr_sk#33] -(55) BroadcastHashJoin [codegen id : 13] +(55) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#26] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(56) Project [codegen id : 13] +(56) Project [codegen id : 2] Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33] Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#32, c_current_addr_sk#33] (57) ReusedExchange [Reuses operator id: 31] Output [1]: [ca_address_sk#34] -(58) BroadcastHashJoin [codegen id : 13] +(58) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_current_addr_sk#33] Right keys [1]: [ca_address_sk#34] Join type: Inner Join condition: None -(59) Project [codegen id : 13] +(59) Project [codegen id : 2] Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33, ca_address_sk#34] (60) ReusedExchange [Reuses operator id: 38] Output [1]: [i_item_sk#35] -(61) BroadcastHashJoin [codegen id : 13] +(61) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#25] Right keys [1]: [i_item_sk#35] Join type: Inner Join condition: None -(62) Project [codegen id : 13] +(62) Project [codegen id : 2] Output [1]: [ss_ext_sales_price#28] Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#35] -(63) HashAggregate [codegen id : 13] +(63) HashAggregate [codegen id : 2] Input [1]: [ss_ext_sales_price#28] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] @@ -366,10 +366,10 @@ Results [1]: [sum#37] Input [1]: [sum#37] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(65) CometColumnarToRow [codegen id : 14] +(65) CometNativeColumnarToRow Input [1]: [sum#37] -(66) HashAggregate [codegen id : 14] +(66) HashAggregate [codegen id : 3] Input [1]: [sum#37] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] @@ -380,11 +380,11 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#38,17,2) AS Input [1]: [total#39] Arguments: IdentityBroadcastMode, [plan_id=8] -(68) BroadcastNestedLoopJoin [codegen id : 15] +(68) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(69) Project [codegen id : 15] +(69) Project [codegen id : 4] Output [3]: [promotions#24, total#39, ((cast(promotions#24 as decimal(15,4)) / cast(total#39 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] Input [2]: [promotions#24, total#39] @@ -392,7 +392,7 @@ Input [2]: [promotions#24, total#39] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometProject (72) +- CometFilter (71) +- CometNativeScan parquet spark_catalog.default.date_dim (70) @@ -413,7 +413,7 @@ Condition : ((((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1 Input [3]: [d_date_sk#14, d_year#41, d_moy#42] Arguments: [d_date_sk#14], [d_date_sk#14] -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (74) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt index 0443f5fc55..025280d4ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt @@ -1,7 +1,7 @@ Project +- BroadcastNestedLoopJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,42 +21,42 @@ Project : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.promotion : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -74,26 +74,26 @@ Project : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/simplified.txt index f5bacac9c6..92507e130e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ -WholeStageCodegen (15) +WholeStageCodegen (4) Project [promotions,total] BroadcastNestedLoopJoin HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -25,61 +25,49 @@ WholeStageCodegen (15) Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_gmt_offset] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometNativeColumnarToRow + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + CometNativeColumnarToRow + CometFilter [c_customer_sk,c_current_addr_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] InputAdapter BroadcastExchange #8 - WholeStageCodegen (14) + WholeStageCodegen (3) HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (13) + WholeStageCodegen (2) HashAggregate [ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/explain.txt index 4f090653c8..29348cf7c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/explain.txt @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt index 7bae7d5724..f359e64d7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ Project : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/simplified.txt index 6ebe25b753..1709804080 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ WholeStageCodegen (2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] CometFilter [s_store_sk,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index 4f090653c8..29348cf7c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 Input [3]: [d_date_sk#14, d_year#15, d_moy#16] Arguments: [d_date_sk#14], [d_date_sk#14] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt index 7bae7d5724..f359e64d7b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt @@ -20,7 +20,7 @@ Project : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6ebe25b753..1709804080 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 @@ -22,12 +22,10 @@ WholeStageCodegen (2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [s_store_sk] #3 CometProject [s_store_sk] CometFilter [s_store_sk,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/explain.txt index 1d9748055e..fc3827793a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (29) +CometNativeColumnarToRow (29) +- CometTakeOrderedAndProject (28) +- CometHashAggregate (27) +- CometExchange (26) @@ -163,6 +163,6 @@ Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt index 52d7828b06..dc88244743 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/simplified.txt index de80c17349..1407717d50 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/simplified.txt @@ -1,31 +1,29 @@ -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 [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 [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] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +CometNativeColumnarToRow + 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 [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 [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] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_site_sk,web_name] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometProject [sm_type] [sm_ship_mode_sk,sm_type] + CometFilter [sm_ship_mode_sk,sm_type] + CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [web_site_sk,web_name] #4 + CometFilter [web_site_sk,web_name] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/explain.txt index 32f70c3d33..d9e5498574 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (30) +- * Project (29) +- * Filter (28) +- Window (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -16,7 +16,7 @@ TakeOrderedAndProject (30) : +- * BroadcastHashJoin Inner BuildRight (12) : :- * Project (10) : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject (30) : : +- Scan parquet spark_catalog.default.store_sales (5) : +- ReusedExchange (11) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -46,7 +46,7 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [2]: [i_item_sk#1, i_manager_id#5] (5) Scan parquet spark_catalog.default.store_sales @@ -68,26 +68,26 @@ Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] (11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#15, d_moy#16] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#13] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16] Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#16] @@ -102,24 +102,24 @@ ReadSchema: struct Input [1]: [s_store_sk#17] Condition : isnotnull(s_store_sk#17) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [s_store_sk#17] (17) BroadcastExchange Input [1]: [s_store_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#11] Right keys [1]: [s_store_sk#17] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16, s_store_sk#17] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] Keys [2]: [i_manager_id#5, d_moy#16] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] @@ -130,10 +130,10 @@ Results [3]: [i_manager_id#5, d_moy#16, sum#19] Input [3]: [i_manager_id#5, d_moy#16, sum#19] Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [3]: [i_manager_id#5, d_moy#16, sum#19] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [3]: [i_manager_id#5, d_moy#16, sum#19] Keys [2]: [i_manager_id#5, d_moy#16] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] @@ -148,18 +148,18 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumn Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] (27) Window Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 7] +(29) Project [codegen id : 4] Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] @@ -171,7 +171,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -192,7 +192,7 @@ Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_moy#16] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt index 0b6c2edaa0..5c7a26057d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject +- Project +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -25,17 +25,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/simplified.txt index 2933149226..7bb18e1026 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/simplified.txt @@ -1,53 +1,47 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manager_id,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometNativeColumnarToRow + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + WholeStageCodegen (3) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_manager_id,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt index e2a0cceb1a..2c40f0d826 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt index 02166879a9..2f7b6145a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index e2a0cceb1a..2c40f0d826 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 02166879a9..2f7b6145a3 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/explain.txt index c021738ca6..cc6e3125df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (180) +CometNativeColumnarToRow (180) +- CometSort (179) +- CometExchange (178) +- CometProject (177) @@ -48,7 +48,7 @@ : : : : : : : : : : : : : : : : : : +- * Filter (3) : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (8) : : : : : : : : : : : : : : : : : +- CometProject (7) : : : : : : : : : : : : : : : : : +- CometFilter (6) : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) @@ -153,7 +153,7 @@ : : : : : : : : : : : : : : : : : +- * Filter (108) : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (113) : : : : : : : : : : : : : : : : +- CometProject (112) : : : : : : : : : : : : : : : : +- CometFilter (111) : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) @@ -215,7 +215,7 @@ Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) CometColumnarToRow +(8) CometNativeColumnarToRow Input [2]: [sr_item_sk#14, sr_ticket_number#15] (9) BroadcastHashJoin [codegen id : 2] @@ -694,7 +694,7 @@ Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] -(113) CometColumnarToRow +(113) CometNativeColumnarToRow Input [2]: [sr_item_sk#111, sr_ticket_number#112] (114) BroadcastHashJoin [codegen id : 4] @@ -971,14 +971,14 @@ Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC 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: [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 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST] -(180) CometColumnarToRow [codegen id : 5] +(180) CometNativeColumnarToRow 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometNativeScan parquet spark_catalog.default.date_dim (181) @@ -994,7 +994,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (184) BroadcastExchange @@ -1003,7 +1003,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (188) -+- * CometColumnarToRow (187) ++- CometNativeColumnarToRow (187) +- CometFilter (186) +- CometNativeScan parquet spark_catalog.default.date_dim (185) @@ -1019,7 +1019,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(187) CometColumnarToRow [codegen id : 1] +(187) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (188) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt index 6e3f37148b..98db64e5b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -49,10 +49,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -171,10 +171,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/simplified.txt index f954616bb5..84648a9f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/simplified.txt @@ -1,206 +1,200 @@ -WholeStageCodegen (5) - 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] - 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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_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] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +CometNativeColumnarToRow + 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] + 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + WholeStageCodegen (2) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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_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] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometNativeScan 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + WholeStageCodegen (4) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #21 + WholeStageCodegen (3) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [d_date_sk,d_year] #12 ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/explain.txt index cbe790cd00..42668314fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/explain.txt @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/simplified.txt index 43527978d8..bfac9da68d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/simplified.txt @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index cbe790cd00..42668314fd 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 @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject 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 43527978d8..bfac9da68d 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 @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/explain.txt index 7cbfb795e8..10f61d3b97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/explain.txt @@ -6,13 +6,13 @@ TakeOrderedAndProject (43) : +- * BroadcastHashJoin Inner BuildRight (23) : :- * Project (17) : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * CometColumnarToRow (3) + : : :- CometNativeColumnarToRow (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.store (1) : : +- BroadcastExchange (15) : : +- * Filter (14) : : +- * HashAggregate (13) - : : +- * CometColumnarToRow (12) + : : +- CometNativeColumnarToRow (12) : : +- CometColumnarExchange (11) : : +- * HashAggregate (10) : : +- * Project (9) @@ -22,18 +22,18 @@ TakeOrderedAndProject (43) : : : +- Scan parquet spark_catalog.default.store_sales (4) : : +- ReusedExchange (7) : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometProject (20) : +- CometFilter (19) : +- CometNativeScan parquet spark_catalog.default.item (18) +- BroadcastExchange (40) +- * Filter (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * HashAggregate (34) - +- * CometColumnarToRow (33) + +- CometNativeColumnarToRow (33) +- CometColumnarExchange (32) +- * HashAggregate (31) +- * Project (30) @@ -55,7 +55,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) CometColumnarToRow [codegen id : 9] +(3) CometNativeColumnarToRow Input [2]: [s_store_sk#1, s_store_name#2] (4) Scan parquet spark_catalog.default.store_sales @@ -66,27 +66,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -(6) Filter [codegen id : 2] +(6) Filter [codegen id : 1] Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) (7) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#8] -(8) BroadcastHashJoin [codegen id : 2] +(8) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#6] Right keys [1]: [d_date_sk#8] Join type: Inner Join condition: None -(9) Project [codegen id : 2] +(9) Project [codegen id : 1] Output [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] -(10) HashAggregate [codegen id : 2] +(10) HashAggregate [codegen id : 1] Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] @@ -97,17 +97,17 @@ Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] -(14) Filter [codegen id : 3] +(14) Filter [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Condition : isnotnull(revenue#12) @@ -115,13 +115,13 @@ Condition : isnotnull(revenue#12) Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 9] +(16) BroadcastHashJoin [codegen id : 6] Left keys [1]: [s_store_sk#1] Right keys [1]: [ss_store_sk#4] Join type: Inner Join condition: None -(17) Project [codegen id : 9] +(17) Project [codegen id : 6] Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] @@ -140,20 +140,20 @@ Condition : isnotnull(i_item_sk#13) Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] Arguments: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18], [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#17, 50, true, false, true) AS i_brand#18] -(21) CometColumnarToRow [codegen id : 4] +(21) CometNativeColumnarToRow Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] (22) BroadcastExchange Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_item_sk#3] Right keys [1]: [i_item_sk#13] Join type: Inner Join condition: None -(24) Project [codegen id : 9] +(24) Project [codegen id : 6] Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] @@ -165,27 +165,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(26) ColumnarToRow [codegen id : 6] +(26) ColumnarToRow [codegen id : 3] Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -(27) Filter [codegen id : 6] +(27) Filter [codegen id : 3] Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] Condition : isnotnull(ss_store_sk#20) (28) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#23] -(29) BroadcastHashJoin [codegen id : 6] +(29) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#22] Right keys [1]: [d_date_sk#23] Join type: Inner Join condition: None -(30) Project [codegen id : 6] +(30) Project [codegen id : 3] Output [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] Input [5]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#23] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 3] Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] Keys [2]: [ss_store_sk#20, ss_item_sk#19] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] @@ -196,17 +196,17 @@ Results [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(33) CometColumnarToRow [codegen id : 7] +(33) CometNativeColumnarToRow Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -(34) HashAggregate [codegen id : 7] +(34) HashAggregate [codegen id : 4] Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] Keys [2]: [ss_store_sk#20, ss_item_sk#19] Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#21))#26] Results [2]: [ss_store_sk#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#21))#26,17,2) AS revenue#27] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 4] Input [2]: [ss_store_sk#20, revenue#27] Keys [1]: [ss_store_sk#20] Functions [1]: [partial_avg(revenue#27)] @@ -217,17 +217,17 @@ Results [3]: [ss_store_sk#20, sum#30, count#31] Input [3]: [ss_store_sk#20, sum#30, count#31] Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [3]: [ss_store_sk#20, sum#30, count#31] -(38) HashAggregate [codegen id : 8] +(38) HashAggregate [codegen id : 5] Input [3]: [ss_store_sk#20, sum#30, count#31] Keys [1]: [ss_store_sk#20] Functions [1]: [avg(revenue#27)] Aggregate Attributes [1]: [avg(revenue#27)#32] Results [2]: [ss_store_sk#20, avg(revenue#27)#32 AS ave#33] -(39) Filter [codegen id : 8] +(39) Filter [codegen id : 5] Input [2]: [ss_store_sk#20, ave#33] Condition : isnotnull(ave#33) @@ -235,13 +235,13 @@ Condition : isnotnull(ave#33) Input [2]: [ss_store_sk#20, ave#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#20] Join type: Inner Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) -(42) Project [codegen id : 9] +(42) Project [codegen id : 6] Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18, ss_store_sk#20, ave#33] @@ -253,7 +253,7 @@ Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -274,7 +274,7 @@ Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_mo Input [2]: [d_date_sk#8, d_month_seq#34] Arguments: [d_date_sk#8], [d_date_sk#8] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt index b1bae81b20..73e3aa7046 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt @@ -5,13 +5,13 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,28 +21,28 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -52,7 +52,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/simplified.txt index 2695c9fb89..98833264f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/simplified.txt @@ -1,24 +1,24 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) + WholeStageCodegen (6) Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] BroadcastHashJoin [ss_item_sk,i_item_sk] Project [s_store_name,ss_store_sk,ss_item_sk,revenue] BroadcastHashJoin [s_store_sk,ss_store_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_store_name] CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #1 - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [revenue] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] Project [ss_item_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -28,37 +28,33 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometNativeColumnarToRow + CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] InputAdapter BroadcastExchange #5 - WholeStageCodegen (8) + WholeStageCodegen (5) Filter [ave] HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_store_sk] #6 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_store_sk,ss_item_sk] #7 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] Project [ss_item_sk,ss_store_sk,ss_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/explain.txt index 8b38832289..98d7e173d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/explain.txt @@ -231,7 +231,7 @@ Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt index 1fc09bd2e3..755be00004 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/simplified.txt index cd1c84b6f2..bfc2fc8751 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 8b38832289..98d7e173d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -231,7 +231,7 @@ Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/extended.txt index 1fc09bd2e3..755be00004 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 cd1c84b6f2..bfc2fc8751 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/explain.txt index 56c5025634..33fb160f49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == TakeOrderedAndProject (56) +- * HashAggregate (55) - +- * CometColumnarToRow (54) + +- CometNativeColumnarToRow (54) +- CometColumnarExchange (53) +- * HashAggregate (52) +- Union (51) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * Project (27) @@ -21,23 +21,23 @@ TakeOrderedAndProject (56) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.web_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) : : : +- ReusedExchange (11) : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) + : : +- CometNativeColumnarToRow (17) : : +- CometProject (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.time_dim (14) : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) + : +- CometNativeColumnarToRow (24) : +- CometProject (23) : +- CometFilter (22) : +- CometNativeScan parquet spark_catalog.default.ship_mode (21) +- * HashAggregate (50) - +- * CometColumnarToRow (49) + +- CometNativeColumnarToRow (49) +- CometColumnarExchange (48) +- * HashAggregate (47) +- * Project (46) @@ -65,10 +65,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) @@ -87,33 +87,33 @@ Condition : isnotnull(w_warehouse_sk#9) Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#14, 2, true, false, true) AS w_state#16, w_country#15] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] (8) BroadcastExchange Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_warehouse_sk#3] Right keys [1]: [w_warehouse_sk#9] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] (11) ReusedExchange [Reuses operator id: 60] Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#7] Right keys [1]: [d_date_sk#17] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] @@ -132,20 +132,20 @@ Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= Input [2]: [t_time_sk#20, t_time#21] Arguments: [t_time_sk#20], [t_time_sk#20] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [t_time_sk#20] (18) BroadcastExchange Input [1]: [t_time_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_time_sk#1] Right keys [1]: [t_time_sk#20] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] @@ -164,24 +164,24 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [1]: [sm_ship_mode_sk#22] (25) BroadcastExchange Input [1]: [sm_ship_mode_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_ship_mode_sk#2] Right keys [1]: [sm_ship_mode_sk#22] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] @@ -192,10 +192,10 @@ Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] @@ -210,66 +210,66 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#176), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 11] +(33) ColumnarToRow [codegen id : 3] Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] -(34) Filter [codegen id : 11] +(34) Filter [codegen id : 3] Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] Condition : ((isnotnull(cs_warehouse_sk#172) AND isnotnull(cs_sold_time_sk#170)) AND isnotnull(cs_ship_mode_sk#171)) (35) ReusedExchange [Reuses operator id: 8] Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -(36) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_warehouse_sk#172] Right keys [1]: [w_warehouse_sk#177] Join type: Inner Join condition: None -(37) Project [codegen id : 11] +(37) Project [codegen id : 3] Output [12]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] (38) ReusedExchange [Reuses operator id: 60] Output [3]: [d_date_sk#184, d_year#185, d_moy#186] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#176] Right keys [1]: [d_date_sk#184] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 3] Output [13]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [15]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] (41) ReusedExchange [Reuses operator id: 18] Output [1]: [t_time_sk#187] -(42) BroadcastHashJoin [codegen id : 11] +(42) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_time_sk#170] Right keys [1]: [t_time_sk#187] Join type: Inner Join condition: None -(43) Project [codegen id : 11] +(43) Project [codegen id : 3] Output [12]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] (44) ReusedExchange [Reuses operator id: 25] Output [1]: [sm_ship_mode_sk#188] -(45) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_ship_mode_sk#171] Right keys [1]: [sm_ship_mode_sk#188] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(46) Project [codegen id : 3] Output [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Input [13]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] -(47) HashAggregate [codegen id : 11] +(47) HashAggregate [codegen id : 3] Input [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] @@ -280,10 +280,10 @@ Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(49) CometColumnarToRow [codegen id : 12] +(49) CometNativeColumnarToRow Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -(50) HashAggregate [codegen id : 12] +(50) HashAggregate [codegen id : 4] Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] @@ -292,7 +292,7 @@ Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county (51) Union -(52) HashAggregate [codegen id : 13] +(52) HashAggregate [codegen id : 5] Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#146, feb_sales#147, mar_sales#148, apr_sales#149, may_sales#150, jun_sales#151, jul_sales#152, aug_sales#153, sep_sales#154, oct_sales#155, nov_sales#156, dec_sales#157, jan_net#158, feb_net#159, mar_net#160, apr_net#161, may_net#162, jun_net#163, jul_net#164, aug_net#165, sep_net#166, oct_net#167, nov_net#168, dec_net#169] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] Functions [36]: [partial_sum(jan_sales#146), partial_sum(feb_sales#147), partial_sum(mar_sales#148), partial_sum(apr_sales#149), partial_sum(may_sales#150), partial_sum(jun_sales#151), partial_sum(jul_sales#152), partial_sum(aug_sales#153), partial_sum(sep_sales#154), partial_sum(oct_sales#155), partial_sum(nov_sales#156), partial_sum(dec_sales#157), partial_sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#158), partial_sum(feb_net#159), partial_sum(mar_net#160), partial_sum(apr_net#161), partial_sum(may_net#162), partial_sum(jun_net#163), partial_sum(jul_net#164), partial_sum(aug_net#165), partial_sum(sep_net#166), partial_sum(oct_net#167), partial_sum(nov_net#168), partial_sum(dec_net#169)] @@ -303,10 +303,10 @@ Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(54) CometColumnarToRow [codegen id : 14] +(54) CometNativeColumnarToRow Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -(55) HashAggregate [codegen id : 14] +(55) HashAggregate [codegen id : 6] Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] Functions [36]: [sum(jan_sales#146), sum(feb_sales#147), sum(mar_sales#148), sum(apr_sales#149), sum(may_sales#150), sum(jun_sales#151), sum(jul_sales#152), sum(aug_sales#153), sum(sep_sales#154), sum(oct_sales#155), sum(nov_sales#156), sum(dec_sales#157), sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#158), sum(feb_net#159), sum(mar_net#160), sum(apr_net#161), sum(may_net#162), sum(jun_net#163), sum(jul_net#164), sum(aug_net#165), sum(sep_net#166), sum(oct_net#167), sum(nov_net#168), sum(dec_net#169)] @@ -321,7 +321,7 @@ Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_w Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (60) -+- * CometColumnarToRow (59) ++- CometNativeColumnarToRow (59) +- CometFilter (58) +- CometNativeScan parquet spark_catalog.default.date_dim (57) @@ -337,7 +337,7 @@ ReadSchema: struct Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) -(59) CometColumnarToRow [codegen id : 1] +(59) CometNativeColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_moy#19] (60) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt index caf49b6d52..3f3c6b8d79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,30 +21,30 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.time_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.ship_mode +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -60,21 +60,21 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.time_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/simplified.txt index 3a1f053d60..7ceb1fb345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/simplified.txt @@ -1,19 +1,19 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) + WholeStageCodegen (6) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) + WholeStageCodegen (5) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] @@ -29,43 +29,35 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometNativeColumnarToRow + CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] InputAdapter ReusedExchange [d_date_sk,d_year,d_moy] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + CometNativeColumnarToRow + CometProject [t_time_sk] + CometFilter [t_time_sk,t_time] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) + CometNativeColumnarToRow + CometProject [sm_ship_mode_sk] + CometFilter [sm_ship_mode_sk,sm_carrier] + CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + WholeStageCodegen (4) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) + WholeStageCodegen (3) HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/explain.txt index 7599e9ecd3..93b64a4b6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/explain.txt @@ -289,7 +289,7 @@ Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -305,7 +305,7 @@ ReadSchema: struct Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_moy#19] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt index ca39e001b0..7e7f77c2c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/simplified.txt index bc6bde24b2..f3122945a6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/simplified.txt @@ -21,11 +21,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 7599e9ecd3..93b64a4b6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -289,7 +289,7 @@ Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -305,7 +305,7 @@ ReadSchema: struct Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#17, d_year#18, d_moy#19] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/extended.txt index ca39e001b0..7e7f77c2c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange 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 bc6bde24b2..f3122945a6 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 @@ -21,11 +21,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/explain.txt index 35fe4c82c3..f5429a1da3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (31) +- * Filter (30) +- Window (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometSort (27) +- CometColumnarExchange (26) +- * HashAggregate (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometColumnarExchange (23) +- * HashAggregate (22) +- * Expand (21) @@ -21,12 +21,12 @@ TakeOrderedAndProject (31) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.store (7) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometProject (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.item (14) @@ -40,23 +40,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 36] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] @@ -75,20 +75,20 @@ Condition : isnotnull(s_store_sk#11) Input [2]: [s_store_sk#11, s_store_id#12] Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [s_store_sk#11, s_store_id#13] (11) BroadcastExchange Input [2]: [s_store_sk#11, s_store_id#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] @@ -107,28 +107,28 @@ Condition : isnotnull(i_item_sk#14) Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] (18) BroadcastExchange Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(21) Expand [codegen id : 4] +(21) Expand [codegen id : 1] Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#21, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 1] Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -139,10 +139,10 @@ Results [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year# Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] Arguments: hashpartitioning(i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometColumnarToRow [codegen id : 5] +(24) CometNativeColumnarToRow Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 2] Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -157,14 +157,14 @@ Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, CometColumna Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] Arguments: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37], [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST] -(28) CometColumnarToRow [codegen id : 6] +(28) CometNativeColumnarToRow Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] (29) Window Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] Arguments: [rank(sumsales#37) windowspecdefinition(i_category#23, sumsales#37 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [i_category#23], [sumsales#37 DESC NULLS LAST] -(30) Filter [codegen id : 7] +(30) Filter [codegen id : 3] Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] Condition : (rk#38 <= 100) @@ -176,7 +176,7 @@ Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -197,7 +197,7 @@ Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1200)) AND (d_mo Input [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt index c3c3850224..27b7a8ecb4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -20,22 +20,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/simplified.txt index 119660fde5..3c4b41657a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/simplified.txt @@ -1,54 +1,46 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) + WholeStageCodegen (3) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt index df94b5d986..e3a82b35ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt @@ -172,7 +172,7 @@ Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -193,7 +193,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt index 708d91d578..0ba8f6b0e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt index e85c243077..d44ea8f6cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index df94b5d986..e3a82b35ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -172,7 +172,7 @@ Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -193,7 +193,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt index 708d91d578..0ba8f6b0e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e85c243077..d44ea8f6cf 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 @@ -22,12 +22,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/explain.txt index 7da946625e..e9ff6848dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (41) :- * Project (37) : +- * BroadcastHashJoin Inner BuildRight (36) : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) + : : +- CometNativeColumnarToRow (29) : : +- CometColumnarExchange (28) : : +- * HashAggregate (27) : : +- * Project (26) @@ -21,21 +21,21 @@ TakeOrderedAndProject (41) : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : : +- ReusedExchange (4) : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) + : : : : +- CometNativeColumnarToRow (10) : : : : +- CometProject (9) : : : : +- CometFilter (8) : : : : +- CometNativeScan parquet spark_catalog.default.store (7) : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) + : : : +- CometNativeColumnarToRow (17) : : : +- CometProject (16) : : : +- CometFilter (15) : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometFilter (22) : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) + : +- CometNativeColumnarToRow (34) : +- CometProject (33) : +- CometFilter (32) : +- CometNativeScan parquet spark_catalog.default.customer (31) @@ -50,23 +50,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#11] -(5) BroadcastHashJoin [codegen id : 5] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(6) Project [codegen id : 5] +(6) Project [codegen id : 1] Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] @@ -85,20 +85,20 @@ Condition : (s_city#13 IN (Midway,Fairview) AND isnotnull(s_store_sk#12)) Input [2]: [s_store_sk#12, s_city#13] Arguments: [s_store_sk#12], [s_store_sk#12] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#12] (11) BroadcastExchange Input [1]: [s_store_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#12] @@ -117,20 +117,20 @@ Condition : (((hd_dep_count#15 = 4) OR (hd_vehicle_count#16 = 3)) AND isnotnull( Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] Arguments: [hd_demo_sk#14], [hd_demo_sk#14] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#14] (18) BroadcastExchange Input [1]: [hd_demo_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#14] @@ -145,24 +145,24 @@ ReadSchema: struct Input [2]: [ca_address_sk#17, ca_city#18] Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_city#18)) -(23) CometColumnarToRow [codegen id : 4] +(23) CometNativeColumnarToRow Input [2]: [ca_address_sk#17, ca_city#18] (24) BroadcastExchange Input [2]: [ca_address_sk#17, ca_city#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(25) BroadcastHashJoin [codegen id : 5] +(25) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_addr_sk#3] Right keys [1]: [ca_address_sk#17] Join type: Inner Join condition: None -(26) Project [codegen id : 5] +(26) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#17, ca_city#18] -(27) HashAggregate [codegen id : 5] +(27) HashAggregate [codegen id : 1] Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] @@ -173,10 +173,10 @@ Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, su Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(29) CometColumnarToRow [codegen id : 8] +(29) CometNativeColumnarToRow Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -(30) HashAggregate [codegen id : 8] +(30) HashAggregate [codegen id : 2] Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] @@ -198,33 +198,33 @@ Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] Arguments: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37], [c_customer_sk#32, c_current_addr_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#37] -(34) CometColumnarToRow [codegen id : 6] +(34) CometNativeColumnarToRow Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] (35) BroadcastExchange Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 2] Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37] Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] (38) ReusedExchange [Reuses operator id: 24] Output [2]: [ca_address_sk#38, ca_city#39] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_current_addr_sk#33] Right keys [1]: [ca_address_sk#38] Join type: Inner Join condition: NOT (ca_city#39 = bought_city#28) -(40) Project [codegen id : 8] +(40) Project [codegen id : 2] Output [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_city#39] @@ -236,7 +236,7 @@ Arguments: 100, [c_last_name#37 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FI Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometNativeScan parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : ((((isnotnull(d_dom#41) AND (d_dom#41 >= 1)) AND (d_dom#41 <= 2)) AN Input [3]: [d_date_sk#11, d_year#40, d_dom#41] Arguments: [d_date_sk#11], [d_date_sk#11] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt index c122bf3803..cebcd85144 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -20,36 +20,36 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/simplified.txt index 05b883b1e6..5a807ddd72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] BroadcastHashJoin [ss_addr_sk,ca_address_sk] @@ -24,44 +24,34 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_city] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometNativeColumnarToRow + CometFilter [ca_address_sk,ca_city] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] InputAdapter BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] InputAdapter ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/explain.txt index bc17101a84..23a09a8dd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/explain.txt @@ -225,7 +225,7 @@ Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/simplified.txt index fdddd82de0..ee6be5a221 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index bc17101a84..23a09a8dd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -225,7 +225,7 @@ Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_tick Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (44) -+- * CometColumnarToRow (43) ++- CometNativeColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) @@ -246,7 +246,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(43) CometColumnarToRow [codegen id : 1] +(43) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (44) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/extended.txt index c842000e11..28f2b6110d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fdddd82de0..ee6be5a221 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/explain.txt index 3641cb51df..4fb599023b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (44) +- * HashAggregate (43) - +- * CometColumnarToRow (42) + +- CometNativeColumnarToRow (42) +- CometColumnarExchange (41) +- * HashAggregate (40) +- * Project (39) @@ -12,7 +12,7 @@ TakeOrderedAndProject (44) : : +- * BroadcastHashJoin LeftAnti BuildRight (24) : : :- * BroadcastHashJoin LeftAnti BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -34,12 +34,12 @@ TakeOrderedAndProject (44) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (30) - : +- * CometColumnarToRow (29) + : +- CometNativeColumnarToRow (29) : +- CometProject (28) : +- CometFilter (27) : +- CometNativeScan parquet spark_catalog.default.customer_address (26) +- BroadcastExchange (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.customer_demographics (33) @@ -56,7 +56,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] (6) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] @@ -86,7 +86,7 @@ Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi @@ -99,19 +99,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] (13) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#7] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#8] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#7] @@ -119,7 +119,7 @@ Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#7] Input [1]: [ws_bill_customer_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#8] Join type: LeftAnti @@ -132,19 +132,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#6)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#10, cs_sold_date_sk#11] (20) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#7] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#11] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#10] Input [3]: [cs_ship_customer_sk#10, cs_sold_date_sk#11, d_date_sk#7] @@ -152,13 +152,13 @@ Input [3]: [cs_ship_customer_sk#10, cs_sold_date_sk#11, d_date_sk#7] Input [1]: [cs_ship_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#10] Join type: LeftAnti Join condition: None -(25) Project [codegen id : 9] +(25) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -177,20 +177,20 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [ca_address_sk#12, ca_state#13] Arguments: [ca_address_sk#12], [ca_address_sk#12] -(29) CometColumnarToRow [codegen id : 7] +(29) CometNativeColumnarToRow Input [1]: [ca_address_sk#12] (30) BroadcastExchange Input [1]: [ca_address_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 9] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#12] Join type: Inner Join condition: None -(32) Project [codegen id : 9] +(32) Project [codegen id : 4] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#12] @@ -209,24 +209,24 @@ Condition : isnotnull(cd_demo_sk#14) Input [6]: [cd_demo_sk#14, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19] Arguments: [cd_demo_sk#14, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23], [cd_demo_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#15, 1, true, false, true) AS cd_gender#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#16, 1, true, false, true) AS cd_marital_status#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#17, 20, true, false, true) AS cd_education_status#22, cd_purchase_estimate#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#19, 10, true, false, true) AS cd_credit_rating#23] -(36) CometColumnarToRow [codegen id : 8] +(36) CometNativeColumnarToRow Input [6]: [cd_demo_sk#14, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] (37) BroadcastExchange Input [6]: [cd_demo_sk#14, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 9] +(38) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#14] Join type: Inner Join condition: None -(39) Project [codegen id : 9] +(39) Project [codegen id : 4] Output [5]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#14, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 4] Input [5]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] Keys [5]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] Functions [1]: [partial_count(1)] @@ -237,10 +237,10 @@ Results [6]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_pur Input [6]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23, count#25] Arguments: hashpartitioning(cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(42) CometColumnarToRow [codegen id : 10] +(42) CometNativeColumnarToRow Input [6]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23, count#25] -(43) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 5] Input [6]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23, count#25] Keys [5]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] Functions [1]: [count(1)] @@ -255,7 +255,7 @@ Arguments: 100, [cd_gender#20 ASC NULLS FIRST, cd_marital_status#21 ASC NULLS FI Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometNativeScan parquet spark_catalog.default.date_dim (45) @@ -276,7 +276,7 @@ Condition : (((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = Input [3]: [d_date_sk#7, d_year#30, d_moy#31] Arguments: [d_date_sk#7], [d_date_sk#7] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt index 821b101f54..f48c67d6e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -21,12 +21,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,7 +37,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -48,17 +48,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/simplified.txt index f8868e38be..e5b4d44c3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -14,13 +14,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -28,17 +28,15 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -49,7 +47,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -60,17 +58,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/explain.txt index 2986d9c0bf..1d19c5c05f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/explain.txt @@ -36,12 +36,12 @@ TakeOrderedAndProject (46) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) + : +- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometFilter (29) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) +- BroadcastExchange (39) - +- * CometColumnarToRow (38) + +- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) @@ -102,7 +102,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -131,7 +131,7 @@ Input [1]: [ws_bill_customer_sk#10] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#10] Join type: LeftAnti @@ -163,13 +163,13 @@ Input [1]: [cs_ship_customer_sk#13] Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#13] Join type: LeftAnti Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -188,20 +188,20 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [ca_address_sk#16] (32) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 5] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(34) Project [codegen id : 5] +(34) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] @@ -220,24 +220,24 @@ Condition : isnotnull(cd_demo_sk#18) Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Arguments: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#26, cd_purchase_estimate#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#23, 10, true, false, true) AS cd_credit_rating#27] -(38) CometColumnarToRow [codegen id : 4] +(38) CometNativeColumnarToRow Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] (39) BroadcastExchange Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 5] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(41) Project [codegen id : 5] +(41) Project [codegen id : 3] Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -(42) HashAggregate [codegen id : 5] +(42) HashAggregate [codegen id : 3] Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Functions [1]: [partial_count(1)] @@ -248,10 +248,10 @@ Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_pur Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) CometColumnarToRow [codegen id : 6] +(44) CometColumnarToRow [codegen id : 4] Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] -(45) HashAggregate [codegen id : 6] +(45) HashAggregate [codegen id : 4] Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Functions [1]: [count(1)] @@ -266,7 +266,7 @@ Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (51) -+- * CometColumnarToRow (50) ++- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) @@ -287,7 +287,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(50) CometColumnarToRow [codegen id : 1] +(50) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt index b4fbb455ef..5a472217df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -49,12 +49,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/simplified.txt index 68bf32c40b..fd8db7bed2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -24,12 +24,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -56,17 +54,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index 2986d9c0bf..1d19c5c05f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -36,12 +36,12 @@ TakeOrderedAndProject (46) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) + : +- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometFilter (29) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) +- BroadcastExchange (39) - +- * CometColumnarToRow (38) + +- CometNativeColumnarToRow (38) +- CometProject (37) +- CometFilter (36) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) @@ -102,7 +102,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -131,7 +131,7 @@ Input [1]: [ws_bill_customer_sk#10] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ws_bill_customer_sk#10] Join type: LeftAnti @@ -163,13 +163,13 @@ Input [1]: [cs_ship_customer_sk#13] Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#1] Right keys [1]: [cs_ship_customer_sk#13] Join type: LeftAnti Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -188,20 +188,20 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [ca_address_sk#16] (32) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 5] +(33) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(34) Project [codegen id : 5] +(34) Project [codegen id : 3] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] @@ -220,24 +220,24 @@ Condition : isnotnull(cd_demo_sk#18) Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] Arguments: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#26, cd_purchase_estimate#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#23, 10, true, false, true) AS cd_credit_rating#27] -(38) CometColumnarToRow [codegen id : 4] +(38) CometNativeColumnarToRow Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] (39) BroadcastExchange Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 5] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(41) Project [codegen id : 5] +(41) Project [codegen id : 3] Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -(42) HashAggregate [codegen id : 5] +(42) HashAggregate [codegen id : 3] Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Functions [1]: [partial_count(1)] @@ -248,10 +248,10 @@ Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_pur Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) CometColumnarToRow [codegen id : 6] +(44) CometColumnarToRow [codegen id : 4] Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] -(45) HashAggregate [codegen id : 6] +(45) HashAggregate [codegen id : 4] Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] Functions [1]: [count(1)] @@ -266,7 +266,7 @@ Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FI Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (51) -+- * CometColumnarToRow (50) ++- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) @@ -287,7 +287,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(50) CometColumnarToRow [codegen id : 1] +(50) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (51) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/extended.txt index b4fbb455ef..5a472217df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -49,12 +49,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt index 68bf32c40b..fd8db7bed2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] CometColumnarToRow InputAdapter CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -24,12 +24,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -56,17 +54,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/explain.txt index 1a5a7efc91..b177b3ca39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * HashAggregate (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * Project (27) @@ -16,18 +16,18 @@ TakeOrderedAndProject (32) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : +- ReusedExchange (11) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.item (14) +- BroadcastExchange (25) - +- * CometColumnarToRow (24) + +- CometNativeColumnarToRow (24) +- CometProject (23) +- CometFilter (22) +- CometNativeScan parquet spark_catalog.default.promotion (21) @@ -41,10 +41,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) @@ -63,33 +63,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -108,20 +108,20 @@ Condition : isnotnull(i_item_sk#15) Input [2]: [i_item_sk#15, i_item_id#16] Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#16, 16, true, false, true) AS i_item_id#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [i_item_sk#15, i_item_id#17] (18) BroadcastExchange Input [2]: [i_item_sk#15, i_item_id#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#15, i_item_id#17] @@ -140,24 +140,24 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] Arguments: [p_promo_sk#18], [p_promo_sk#18] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [1]: [p_promo_sk#18] (25) BroadcastExchange Input [1]: [p_promo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_promo_sk#3] Right keys [1]: [p_promo_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17, p_promo_sk#18] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] Keys [1]: [i_item_id#17] Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] @@ -168,10 +168,10 @@ Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34 Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] Keys [1]: [i_item_id#17] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] @@ -186,7 +186,7 @@ Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#45] Arguments: [d_date_sk#14], [d_date_sk#14] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt index f40fdbe4e3..b692dfa9be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -16,27 +16,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/simplified.txt index bb670b4a73..d7988608ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] BroadcastHashJoin [ss_promo_sk,p_promo_sk] @@ -20,35 +20,27 @@ TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] + CometNativeColumnarToRow + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_email,p_channel_event] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/explain.txt index 0e32283618..01acebb66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt index 655f651a41..3aa5ad7551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/simplified.txt index a15bcd5c0d..707bfe627e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/simplified.txt @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 0e32283618..01acebb66d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -181,7 +181,7 @@ Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/extended.txt index 655f651a41..3aa5ad7551 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a15bcd5c0d..707bfe627e 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 @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/explain.txt index f185af2c42..e38f53475b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (47) +- * Project (46) +- Window (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometSort (43) +- CometColumnarExchange (42) +- * HashAggregate (41) - +- * CometColumnarToRow (40) + +- CometNativeColumnarToRow (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * Expand (37) @@ -21,7 +21,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (34) +- * Project (33) +- * BroadcastHashJoin LeftSemi BuildRight (32) - :- * CometColumnarToRow (9) + :- CometNativeColumnarToRow (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.store (7) +- BroadcastExchange (31) @@ -30,7 +30,7 @@ TakeOrderedAndProject (47) +- Window (28) +- * Sort (27) +- * HashAggregate (26) - +- * CometColumnarToRow (25) + +- CometNativeColumnarToRow (25) +- CometColumnarExchange (24) +- * HashAggregate (23) +- * Project (22) @@ -41,7 +41,7 @@ TakeOrderedAndProject (47) : : +- * ColumnarToRow (11) : : +- Scan parquet spark_catalog.default.store_sales (10) : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometProject (15) : +- CometFilter (14) : +- CometNativeScan parquet spark_catalog.default.store (13) @@ -56,23 +56,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 5] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -87,7 +87,7 @@ ReadSchema: struct Input [3]: [s_store_sk#6, s_county#7, s_state#8] Condition : isnotnull(s_store_sk#6) -(9) CometColumnarToRow [codegen id : 7] +(9) CometNativeColumnarToRow Input [3]: [s_store_sk#6, s_county#7, s_state#8] (10) Scan parquet spark_catalog.default.store_sales @@ -98,10 +98,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Condition : isnotnull(ss_store_sk#9) @@ -120,37 +120,37 @@ Condition : isnotnull(s_store_sk#6) Input [2]: [s_store_sk#6, s_state#8] Arguments: [s_store_sk#6, s_state#12], [s_store_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#12] -(16) CometColumnarToRow [codegen id : 2] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#6, s_state#12] (17) BroadcastExchange Input [2]: [s_store_sk#6, s_state#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#9] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#12] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#6, s_state#12] (20) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#5] -(21) BroadcastHashJoin [codegen id : 4] +(21) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#11] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(22) Project [codegen id : 4] +(22) Project [codegen id : 1] Output [2]: [ss_net_profit#10, s_state#12] Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#12, d_date_sk#5] -(23) HashAggregate [codegen id : 4] +(23) HashAggregate [codegen id : 1] Input [2]: [ss_net_profit#10, s_state#12] Keys [1]: [s_state#12] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] @@ -161,17 +161,17 @@ Results [2]: [s_state#12, sum#14] Input [2]: [s_state#12, sum#14] Arguments: hashpartitioning(s_state#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(25) CometColumnarToRow [codegen id : 5] +(25) CometNativeColumnarToRow Input [2]: [s_state#12, sum#14] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 2] Input [2]: [s_state#12, sum#14] Keys [1]: [s_state#12] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#15] Results [3]: [s_state#12, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#15,17,2) AS _w0#16, s_state#12] -(27) Sort [codegen id : 5] +(27) Sort [codegen id : 2] Input [3]: [s_state#12, _w0#16, s_state#12] Arguments: [s_state#12 ASC NULLS FIRST, _w0#16 DESC NULLS LAST], false, 0 @@ -179,11 +179,11 @@ Arguments: [s_state#12 ASC NULLS FIRST, _w0#16 DESC NULLS LAST], false, 0 Input [3]: [s_state#12, _w0#16, s_state#12] Arguments: [rank(_w0#16) windowspecdefinition(s_state#12, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#12], [_w0#16 DESC NULLS LAST] -(29) Filter [codegen id : 6] +(29) Filter [codegen id : 3] Input [4]: [s_state#12, _w0#16, s_state#12, ranking#17] Condition : (ranking#17 <= 5) -(30) Project [codegen id : 6] +(30) Project [codegen id : 3] Output [1]: [s_state#12] Input [4]: [s_state#12, _w0#16, s_state#12, ranking#17] @@ -191,13 +191,13 @@ Input [4]: [s_state#12, _w0#16, s_state#12, ranking#17] Input [1]: [s_state#12] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] Right keys [1]: [s_state#12] Join type: LeftSemi Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 4] Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#18] Input [3]: [s_store_sk#6, s_county#7, s_state#8] @@ -205,21 +205,21 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] Input [3]: [s_store_sk#6, s_county#7, s_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(35) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(36) Project [codegen id : 8] +(36) Project [codegen id : 5] Output [3]: [ss_net_profit#2, s_state#18, s_county#7] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#18] -(37) Expand [codegen id : 8] +(37) Expand [codegen id : 5] Input [3]: [ss_net_profit#2, s_state#18, s_county#7] Arguments: [[ss_net_profit#2, s_state#18, s_county#7, 0], [ss_net_profit#2, s_state#18, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] -(38) HashAggregate [codegen id : 8] +(38) HashAggregate [codegen id : 5] Input [4]: [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] @@ -230,10 +230,10 @@ Results [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometColumnarToRow [codegen id : 9] +(40) CometNativeColumnarToRow Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 6] Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] @@ -248,14 +248,14 @@ Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] Arguments: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] -(44) CometColumnarToRow [codegen id : 10] +(44) CometNativeColumnarToRow Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] (45) Window Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] -(46) Project [codegen id : 11] +(46) Project [codegen id : 7] Output [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] Input [8]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] @@ -267,7 +267,7 @@ Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometNativeScan parquet spark_catalog.default.date_dim (48) @@ -288,7 +288,7 @@ Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#31] Arguments: [d_date_sk#5], [d_date_sk#5] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt index 4abbd20203..546d2cc0bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -18,19 +18,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange @@ -39,7 +39,7 @@ TakeOrderedAndProject +- Window +- Sort +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -51,12 +51,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/simplified.txt index 62c46296ca..a3534ccf1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/simplified.txt @@ -1,79 +1,73 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) + WholeStageCodegen (7) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (9) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (6) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + InputAdapter + CometNativeColumnarToRow + CometFilter [s_store_sk,s_county,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (2) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state] #6 + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt index d3610af0f8..a797b30715 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject (48) +- BroadcastExchange (35) +- * Project (34) +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (12) + :- CometNativeColumnarToRow (12) : +- CometFilter (11) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) +- BroadcastExchange (32) @@ -103,7 +103,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -271,7 +271,7 @@ Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -292,7 +292,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt index 6a918ac127..fa1bf8daa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -29,7 +29,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt index d6ba61a844..9150a01fba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -41,8 +39,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index d3610af0f8..a797b30715 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject (48) +- BroadcastExchange (35) +- * Project (34) +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (12) + :- CometNativeColumnarToRow (12) : +- CometFilter (11) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) +- BroadcastExchange (32) @@ -103,7 +103,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -271,7 +271,7 @@ Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -292,7 +292,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt index 6a918ac127..fa1bf8daa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -29,7 +29,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange 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 d6ba61a844..9150a01fba 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -41,8 +39,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/explain.txt index 7c4f959123..81be326835 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (40) +CometNativeColumnarToRow (40) +- CometSort (39) +- CometColumnarExchange (38) +- * HashAggregate (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometColumnarExchange (35) +- * HashAggregate (34) +- * Project (33) @@ -11,7 +11,7 @@ :- * Project (26) : +- * BroadcastHashJoin Inner BuildLeft (25) : :- BroadcastExchange (5) - : : +- * CometColumnarToRow (4) + : : +- CometNativeColumnarToRow (4) : : +- CometProject (3) : : +- CometFilter (2) : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -35,7 +35,7 @@ : : +- Scan parquet spark_catalog.default.store_sales (18) : +- ReusedExchange (21) +- BroadcastExchange (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.time_dim (27) @@ -56,7 +56,7 @@ Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull( Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#3, 50, true, false, true) AS i_brand#5] -(4) CometColumnarToRow [codegen id : 1] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] (5) BroadcastExchange @@ -71,23 +71,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 3] +(7) ColumnarToRow [codegen id : 1] Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -(8) Filter [codegen id : 3] +(8) Filter [codegen id : 1] Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) (9) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#11] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 1] Output [3]: [ws_ext_sales_price#8 AS ext_price#12, ws_item_sk#7 AS sold_item_sk#13, ws_sold_time_sk#6 AS time_sk#14] Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] @@ -99,23 +99,23 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 5] +(13) ColumnarToRow [codegen id : 2] Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -(14) Filter [codegen id : 5] +(14) Filter [codegen id : 2] Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] Condition : (isnotnull(cs_item_sk#16) AND isnotnull(cs_sold_time_sk#15)) (15) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#19] -(16) BroadcastHashJoin [codegen id : 5] +(16) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#18] Right keys [1]: [d_date_sk#19] Join type: Inner Join condition: None -(17) Project [codegen id : 5] +(17) Project [codegen id : 2] Output [3]: [cs_ext_sales_price#17 AS ext_price#20, cs_item_sk#16 AS sold_item_sk#21, cs_sold_time_sk#15 AS time_sk#22] Input [5]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#19] @@ -127,35 +127,35 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 7] +(19) ColumnarToRow [codegen id : 3] Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(20) Filter [codegen id : 7] +(20) Filter [codegen id : 3] Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) (21) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#27] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 3] Output [3]: [ss_ext_sales_price#25 AS ext_price#28, ss_item_sk#24 AS sold_item_sk#29, ss_sold_time_sk#23 AS time_sk#30] Input [5]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] (24) Union -(25) BroadcastHashJoin [codegen id : 9] +(25) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] Right keys [1]: [sold_item_sk#13] Join type: Inner Join condition: None -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [4]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#12, sold_item_sk#13, time_sk#14] @@ -174,24 +174,24 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] Arguments: [t_time_sk#31, t_hour#32, t_minute#33], [t_time_sk#31, t_hour#32, t_minute#33] -(30) CometColumnarToRow [codegen id : 8] +(30) CometNativeColumnarToRow Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] (31) BroadcastExchange Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [time_sk#14] Right keys [1]: [t_time_sk#31] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] Input [7]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14, t_time_sk#31, t_hour#32, t_minute#33] -(34) HashAggregate [codegen id : 9] +(34) HashAggregate [codegen id : 4] Input [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] Functions [1]: [partial_sum(UnscaledValue(ext_price#12))] @@ -202,10 +202,10 @@ Results [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(36) CometColumnarToRow [codegen id : 10] +(36) CometNativeColumnarToRow Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] -(37) HashAggregate [codegen id : 10] +(37) HashAggregate [codegen id : 5] Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] Functions [1]: [sum(UnscaledValue(ext_price#12))] @@ -220,14 +220,14 @@ Arguments: rangepartitioning(ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] Arguments: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40], [ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 11] +(40) CometNativeColumnarToRow Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.date_dim (41) @@ -248,7 +248,7 @@ Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 11 Input [3]: [d_date_sk#11, d_year#41, d_moy#42] Arguments: [d_date_sk#11], [d_date_sk#11] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt index 427d3518dc..23844f41f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ CometColumnarToRow :- Project : +- BroadcastHashJoin : :- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -22,12 +22,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,12 +49,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/simplified.txt index 838a3e6604..9fdaac52c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/simplified.txt @@ -1,71 +1,63 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 - WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] +CometNativeColumnarToRow + CometSort [brand_id,brand,t_hour,t_minute,ext_price] + CometColumnarExchange [ext_price,brand_id] #1 + WholeStageCodegen (5) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + WholeStageCodegen (4) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] + Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] + BroadcastHashJoin [time_sk,t_time_sk] + Project [i_brand_id,i_brand,ext_price,time_sk] + BroadcastHashJoin [i_item_sk,sold_item_sk] + InputAdapter + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + InputAdapter + Union + WholeStageCodegen (1) + Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk,ws_sold_time_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (2) + Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk,cs_sold_time_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_time_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (3) + Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_sold_time_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + Scan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/explain.txt index bb217d6417..e18049c7aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/explain.txt @@ -216,7 +216,7 @@ Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11 Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt index 25ea15c536..12dc136fee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/simplified.txt index 63e7c353ac..0c925fadcb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index bb217d6417..e18049c7aa 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 @@ -216,7 +216,7 @@ Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11 Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/extended.txt index 25ea15c536..12dc136fee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 63e7c353ac..0c925fadcb 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 @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/explain.txt index ed3a4a7aea..6b720bc351 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (70) +CometNativeColumnarToRow (70) +- CometTakeOrderedAndProject (69) +- CometHashAggregate (68) +- CometExchange (67) @@ -30,38 +30,38 @@ : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) + : : : : : : : : : +- CometNativeColumnarToRow (6) : : : : : : : : : +- CometFilter (5) : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) + : : : : : : : : +- CometNativeColumnarToRow (12) : : : : : : : : +- CometFilter (11) : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometNativeColumnarToRow (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) + : : : : : : +- CometNativeColumnarToRow (25) : : : : : : +- CometProject (24) : : : : : : +- CometFilter (23) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) + : : : : : +- CometNativeColumnarToRow (32) : : : : : +- CometProject (31) : : : : : +- CometFilter (30) : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) : : : : +- ReusedExchange (36) : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) + : : : +- CometNativeColumnarToRow (41) : : : +- CometFilter (40) : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) + : : +- CometNativeColumnarToRow (47) : : +- CometFilter (46) : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometFilter (52) : +- CometNativeScan parquet spark_catalog.default.promotion (51) +- CometSort (63) @@ -79,10 +79,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 10] +(2) ColumnarToRow [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -(3) Filter [codegen id : 10] +(3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) @@ -98,20 +98,20 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] (13) BroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 10] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#11] Right keys [1]: [w_warehouse_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 10] +(15) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] @@ -154,20 +154,20 @@ ReadSchema: struct Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) CometColumnarToRow [codegen id : 3] +(18) CometNativeColumnarToRow Input [2]: [i_item_sk#16, i_item_desc#17] (19) BroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 10] +(20) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#4] Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(21) Project [codegen id : 10] +(21) Project [codegen id : 1] Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] @@ -186,20 +186,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) CometColumnarToRow [codegen id : 4] +(25) CometNativeColumnarToRow Input [1]: [cd_demo_sk#18] (26) BroadcastExchange Input [1]: [cd_demo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 10] +(27) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(28) Project [codegen id : 10] +(28) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] @@ -218,33 +218,33 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) CometColumnarToRow [codegen id : 5] +(32) CometNativeColumnarToRow Input [1]: [hd_demo_sk#20] (33) BroadcastExchange Input [1]: [hd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_hdemo_sk#3] Right keys [1]: [hd_demo_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 10] +(35) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] (36) ReusedExchange [Reuses operator id: 75] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(37) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#8] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(38) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] @@ -259,20 +259,20 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_week_seq#26] Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) -(41) CometColumnarToRow [codegen id : 7] +(41) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_week_seq#26] (42) BroadcastExchange Input [2]: [d_date_sk#25, d_week_seq#26] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 10] +(43) BroadcastHashJoin [codegen id : 1] Left keys [2]: [d_week_seq#24, inv_date_sk#13] Right keys [2]: [d_week_seq#26, d_date_sk#25] Join type: Inner Join condition: None -(44) Project [codegen id : 10] +(44) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] @@ -287,20 +287,20 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_date#28] Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) -(47) CometColumnarToRow [codegen id : 8] +(47) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_date#28] (48) BroadcastExchange Input [2]: [d_date_sk#27, d_date#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: (d_date#28 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(50) Project [codegen id : 1] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] @@ -315,20 +315,20 @@ ReadSchema: struct Input [1]: [p_promo_sk#29] Condition : isnotnull(p_promo_sk#29) -(53) CometColumnarToRow [codegen id : 9] +(53) CometNativeColumnarToRow Input [1]: [p_promo_sk#29] (54) BroadcastExchange Input [1]: [p_promo_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(55) BroadcastHashJoin [codegen id : 10] +(55) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_promo_sk#5] Right keys [1]: [p_promo_sk#29] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(56) Project [codegen id : 1] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] @@ -390,14 +390,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -(70) CometColumnarToRow [codegen id : 11] +(70) CometNativeColumnarToRow Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (75) -+- * CometColumnarToRow (74) ++- CometNativeColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -418,7 +418,7 @@ Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) CometColumnarToRow [codegen id : 1] +(74) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt index 081972705a..62c2b17b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -30,47 +30,47 @@ CometColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/simplified.txt index 025217a607..edcf2b00f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/simplified.txt @@ -1,107 +1,87 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - 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] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + 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] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + BroadcastExchange #6 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + BroadcastExchange #9 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + BroadcastExchange #10 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + CometNativeColumnarToRow + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/explain.txt index bf5bcb475b..1c2d72fad3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/simplified.txt index 678f8c67df..549469c03b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index bf5bcb475b..1c2d72fad3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 678f8c67df..549469c03b 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 @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/explain.txt index 7fe1aa758c..fcc1a09c69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometSort (34) +- CometColumnarExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) :- * Filter (25) : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -20,17 +20,17 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.customer (26) @@ -44,23 +44,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] @@ -79,20 +79,20 @@ Condition : (s_county#9 IN (Williamson County,Franklin Parish,Bronx County,Orang Input [2]: [s_store_sk#8, s_county#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (11) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] @@ -111,24 +111,24 @@ Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apac Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#10] (18) BroadcastExchange Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] @@ -139,17 +139,17 @@ Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) @@ -168,20 +168,20 @@ Condition : isnotnull(c_customer_sk#18) Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] (30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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] @@ -193,14 +193,14 @@ Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, Co 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: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (40) -+- * CometColumnarToRow (39) ++- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometNativeScan parquet spark_catalog.default.date_dim (36) @@ -221,7 +221,7 @@ Condition : ((((isnotnull(d_dom#28) AND (d_dom#28 >= 1)) AND (d_dom#28 <= 2)) AN Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(39) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (40) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt index 9e4596c12c..40230582bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- BroadcastHashJoin :- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,27 +19,27 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/simplified.txt index 4d2a6a74ab..ef3860150b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/simplified.txt @@ -1,59 +1,49 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] +CometNativeColumnarToRow + CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + CometColumnarExchange [cnt] #1 + WholeStageCodegen (2) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/explain.txt index 346e110371..667cf5d6a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/simplified.txt index 227dd00f2b..133182ebde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 346e110371..667cf5d6a8 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 @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 227dd00f2b..133182ebde 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/explain.txt index c019668fc8..3ce14f881d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/explain.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject (79) : :- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -25,14 +25,14 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -44,14 +44,14 @@ TakeOrderedAndProject (79) : +- BroadcastExchange (56) : +- * Filter (55) : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (47) : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) + : : :- CometNativeColumnarToRow (41) : : : +- CometProject (40) : : : +- CometFilter (39) : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -62,14 +62,14 @@ TakeOrderedAndProject (79) : +- ReusedExchange (48) +- BroadcastExchange (76) +- * HashAggregate (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometColumnarExchange (73) +- * HashAggregate (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) :- * Project (68) : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) + : :- CometNativeColumnarToRow (62) : : +- CometProject (61) : : +- CometFilter (60) : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -95,7 +95,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] (5) Scan parquet spark_catalog.default.store_sales @@ -117,30 +117,30 @@ Condition : isnotnull(ss_customer_sk#8) Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] (11) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#12, d_year#13] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] @@ -151,17 +151,17 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) @@ -180,7 +180,7 @@ Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] (23) Scan parquet spark_catalog.default.store_sales @@ -191,10 +191,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Condition : isnotnull(ss_customer_sk#23) @@ -202,30 +202,30 @@ Condition : isnotnull(ss_customer_sk#23) Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#19] Right keys [1]: [ss_customer_sk#23] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#27, d_year#28] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#25] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] @@ -236,10 +236,10 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] @@ -250,7 +250,7 @@ Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_firs Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#31] Join type: Inner @@ -271,7 +271,7 @@ Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] (42) Scan parquet spark_catalog.default.web_sales @@ -282,10 +282,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Condition : isnotnull(ws_bill_customer_sk#42) @@ -293,30 +293,30 @@ Condition : isnotnull(ws_bill_customer_sk#42) Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#35] Right keys [1]: [ws_bill_customer_sk#42] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#45, d_year#46] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#44] Right keys [1]: [d_date_sk#45] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] @@ -327,17 +327,17 @@ Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum# Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#49] Results [2]: [c_customer_id#39 AS customer_id#50, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#49,17,2) AS year_total#51] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#50, year_total#51] Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) @@ -345,13 +345,13 @@ Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) Input [2]: [customer_id#50, year_total#51] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#50] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(58) Project [codegen id : 12] Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51] Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#50, year_total#51] @@ -370,7 +370,7 @@ Condition : (isnotnull(c_customer_sk#52) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] Arguments: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#54, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#55, 30, true, false, true) AS c_last_name#41] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [4]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41] (63) Scan parquet spark_catalog.default.web_sales @@ -381,10 +381,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] Condition : isnotnull(ws_bill_customer_sk#56) @@ -392,30 +392,30 @@ Condition : isnotnull(ws_bill_customer_sk#56) Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#52] Right keys [1]: [ws_bill_customer_sk#56] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58] Input [7]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] (69) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#59, d_year#60] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#58] Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58, d_date_sk#59, d_year#60] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#57))] @@ -426,10 +426,10 @@ Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum# Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] Functions [1]: [sum(UnscaledValue(ws_net_paid#57))] @@ -440,13 +440,13 @@ Results [2]: [c_customer_id#39 AS customer_id#63, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#63, year_total#64] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#63] Join type: Inner Join condition: (CASE WHEN (year_total#51 > 0.00) THEN (year_total#64 / year_total#51) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) -(78) Project [codegen id : 16] +(78) Project [codegen id : 12] Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51, customer_id#63, year_total#64] @@ -458,7 +458,7 @@ Arguments: 100, [customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometFilter (81) +- CometNativeScan parquet spark_catalog.default.date_dim (80) @@ -474,7 +474,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (83) BroadcastExchange @@ -483,7 +483,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (87) -+- * CometColumnarToRow (86) ++- CometNativeColumnarToRow (86) +- CometFilter (85) +- CometNativeScan parquet spark_catalog.default.date_dim (84) @@ -499,7 +499,7 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_year#28] Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) -(86) CometColumnarToRow [codegen id : 1] +(86) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_year#28] (87) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt index 0d59f7b833..5d218998be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -23,23 +23,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -49,24 +49,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -76,19 +76,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -98,7 +98,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/simplified.txt index f488e18c7c..01e22b7b4a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -7,17 +7,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] @@ -30,69 +30,65 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -102,25 +98,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/explain.txt index 649b5dc4d5..046c271650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_year#26] Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_year#26] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/simplified.txt index 72e39422e0..9ec698fdca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/simplified.txt @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index 649b5dc4d5..046c271650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_year#26] Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_year#26] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 72e39422e0..9ec698fdca 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 @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/explain.txt index abecd7ac24..d4b4089cf1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (127) +CometNativeColumnarToRow (127) +- CometTakeOrderedAndProject (126) +- CometProject (125) +- CometSortMergeJoin (124) @@ -25,7 +25,7 @@ : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -136,10 +136,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Condition : isnotnull(cs_item_sk#1) @@ -158,33 +158,33 @@ Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarc Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (8) BroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (11) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#13, d_year#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] @@ -236,36 +236,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(24) ColumnarToRow [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) (26) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#22] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] (29) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#32, d_year#33] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] @@ -317,36 +317,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 9] +(42) ColumnarToRow [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -(43) Filter [codegen id : 9] +(43) Filter [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Condition : isnotnull(ws_item_sk#41) (44) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -(45) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_item_sk#41] Right keys [1]: [i_item_sk#46] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(46) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] (47) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#51, d_year#52] -(48) BroadcastHashJoin [codegen id : 9] +(48) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#45] Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(49) Project [codegen id : 9] +(49) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] @@ -443,36 +443,36 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 12] +(70) ColumnarToRow [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -(71) Filter [codegen id : 12] +(71) Filter [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] Condition : isnotnull(cs_item_sk#64) (72) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -(73) BroadcastHashJoin [codegen id : 12] +(73) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_item_sk#64] Right keys [1]: [i_item_sk#70] Join type: Inner Join condition: None -(74) Project [codegen id : 12] +(74) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] (75) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#75, d_year#76] -(76) BroadcastHashJoin [codegen id : 12] +(76) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#68] Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(77) Project [codegen id : 12] +(77) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] @@ -508,36 +508,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 15] +(85) ColumnarToRow [codegen id : 5] Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -(86) Filter [codegen id : 15] +(86) Filter [codegen id : 5] Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] Condition : isnotnull(ss_item_sk#81) (87) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] -(88) BroadcastHashJoin [codegen id : 15] +(88) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#81] Right keys [1]: [i_item_sk#86] Join type: Inner Join condition: None -(89) Project [codegen id : 15] +(89) Project [codegen id : 5] Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] Input [10]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] (90) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#91, d_year#92] -(91) BroadcastHashJoin [codegen id : 15] +(91) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#85] Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(92) Project [codegen id : 15] +(92) Project [codegen id : 5] Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] Input [11]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_date_sk#91, d_year#92] @@ -573,36 +573,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 18] +(100) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -(101) Filter [codegen id : 18] +(101) Filter [codegen id : 6] Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] Condition : isnotnull(ws_item_sk#97) (102) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] -(103) BroadcastHashJoin [codegen id : 18] +(103) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#97] Right keys [1]: [i_item_sk#102] Join type: Inner Join condition: None -(104) Project [codegen id : 18] +(104) Project [codegen id : 6] Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] Input [10]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] (105) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#107, d_year#108] -(106) BroadcastHashJoin [codegen id : 18] +(106) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#101] Right keys [1]: [d_date_sk#107] Join type: Inner Join condition: None -(107) Project [codegen id : 18] +(107) Project [codegen id : 6] Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] Input [11]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_date_sk#107, d_year#108] @@ -688,14 +688,14 @@ Arguments: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#120 ASC NULLS FIRST], output=[prev_year#116,year#117,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#118,curr_yr_cnt#119,sales_cnt_diff#120,sales_amt_diff#121]), [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], 100, 0, [sales_cnt_diff#120 ASC NULLS FIRST], [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] -(127) CometColumnarToRow [codegen id : 19] +(127) CometNativeColumnarToRow Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (131) -+- * CometColumnarToRow (130) ++- CometNativeColumnarToRow (130) +- CometFilter (129) +- CometNativeScan parquet spark_catalog.default.date_dim (128) @@ -711,7 +711,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (131) BroadcastExchange @@ -724,7 +724,7 @@ Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN d Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 BroadcastExchange (135) -+- * CometColumnarToRow (134) ++- CometNativeColumnarToRow (134) +- CometFilter (133) +- CometNativeScan parquet spark_catalog.default.date_dim (132) @@ -740,7 +740,7 @@ ReadSchema: struct Input [2]: [d_date_sk#75, d_year#76] Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) -(134) CometColumnarToRow [codegen id : 1] +(134) CometNativeColumnarToRow Input [2]: [d_date_sk#75, d_year#76] (135) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt index 9f6954be38..42106a0ffd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometSortMergeJoin @@ -25,16 +25,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -55,12 +55,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -81,12 +81,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -117,16 +117,16 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -147,12 +147,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -173,12 +173,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/simplified.txt index aaf98f4b09..84b015b466 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/simplified.txt @@ -1,173 +1,165 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - 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 [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 [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] +CometNativeColumnarToRow + CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + 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 [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 [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] - 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] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometExchange [cr_order_number,cr_item_sk] #7 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (2) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometExchange [sr_ticket_number,sr_item_sk] #9 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (3) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometExchange [wr_order_number,wr_item_sk] #11 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 + 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] #14 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] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (4) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (5) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (6) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/explain.txt index 7afb1e3be6..0ea1f4907b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/simplified.txt index d0520c8b63..0e2370eea4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/simplified.txt @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 7afb1e3be6..0ea1f4907b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange 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 d0520c8b63..0e2370eea4 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 @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/explain.txt index f04f5a8a5a..435ede764a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometTakeOrderedAndProject (34) +- CometHashAggregate (33) +- CometExchange (32) @@ -195,6 +195,6 @@ Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/extended.txt index cbb627c528..cfa19a28b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/simplified.txt index abb482c3d5..f502dc6f07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/simplified.txt @@ -1,37 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - 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 [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] - CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] - CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedExchange [i_item_sk,i_category] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] - CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] - CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedExchange [i_item_sk,i_category] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #3 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + 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 [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] + CometProject [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_category] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastExchange [i_item_sk,i_category] #2 + CometProject [i_category] [i_item_sk,i_category] + CometFilter [i_item_sk,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_category] + CometFilter [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + CometProject [cs_ship_addr_sk,cs_ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category,d_date_sk,d_year,d_qoy] + CometProject [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_category] + CometFilter [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedExchange [i_item_sk,i_category] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/explain.txt index ae530b4900..d68b43b40a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (92) +- * HashAggregate (91) - +- * CometColumnarToRow (90) + +- CometNativeColumnarToRow (90) +- CometColumnarExchange (89) +- * HashAggregate (88) +- * Expand (87) @@ -9,7 +9,7 @@ TakeOrderedAndProject (92) :- * Project (32) : +- * BroadcastHashJoin LeftOuter BuildRight (31) : :- * HashAggregate (16) - : : +- * CometColumnarToRow (15) + : : +- CometNativeColumnarToRow (15) : : +- CometColumnarExchange (14) : : +- * HashAggregate (13) : : +- * Project (12) @@ -21,12 +21,12 @@ TakeOrderedAndProject (92) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (10) - : : +- * CometColumnarToRow (9) + : : +- CometNativeColumnarToRow (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (30) : +- * HashAggregate (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometColumnarExchange (27) : +- * HashAggregate (26) : +- * Project (25) @@ -42,7 +42,7 @@ TakeOrderedAndProject (92) : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) : :- BroadcastExchange (42) : : +- * HashAggregate (41) - : : +- * CometColumnarToRow (40) + : : +- CometNativeColumnarToRow (40) : : +- CometColumnarExchange (39) : : +- * HashAggregate (38) : : +- * Project (37) @@ -51,7 +51,7 @@ TakeOrderedAndProject (92) : : : +- Scan parquet spark_catalog.default.catalog_sales (33) : : +- ReusedExchange (35) : +- * HashAggregate (51) - : +- * CometColumnarToRow (50) + : +- CometNativeColumnarToRow (50) : +- CometColumnarExchange (49) : +- * HashAggregate (48) : +- * Project (47) @@ -62,7 +62,7 @@ TakeOrderedAndProject (92) +- * Project (85) +- * BroadcastHashJoin LeftOuter BuildRight (84) :- * HashAggregate (69) - : +- * CometColumnarToRow (68) + : +- CometNativeColumnarToRow (68) : +- CometColumnarExchange (67) : +- * HashAggregate (66) : +- * Project (65) @@ -74,12 +74,12 @@ TakeOrderedAndProject (92) : : : +- Scan parquet spark_catalog.default.web_sales (54) : : +- ReusedExchange (57) : +- BroadcastExchange (63) - : +- * CometColumnarToRow (62) + : +- CometNativeColumnarToRow (62) : +- CometFilter (61) : +- CometNativeScan parquet spark_catalog.default.web_page (60) +- BroadcastExchange (83) +- * HashAggregate (82) - +- * CometColumnarToRow (81) + +- CometNativeColumnarToRow (81) +- CometColumnarExchange (80) +- * HashAggregate (79) +- * Project (78) @@ -101,23 +101,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] @@ -132,24 +132,24 @@ ReadSchema: struct Input [1]: [s_store_sk#7] Condition : isnotnull(s_store_sk#7) -(9) CometColumnarToRow [codegen id : 2] +(9) CometNativeColumnarToRow Input [1]: [s_store_sk#7] (10) BroadcastExchange Input [1]: [s_store_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(12) Project [codegen id : 3] +(12) Project [codegen id : 1] Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Keys [1]: [s_store_sk#7] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] @@ -160,10 +160,10 @@ Results [3]: [s_store_sk#7, sum#10, sum#11] Input [3]: [s_store_sk#7, sum#10, sum#11] Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(15) CometColumnarToRow [codegen id : 8] +(15) CometNativeColumnarToRow Input [3]: [s_store_sk#7, sum#10, sum#11] -(16) HashAggregate [codegen id : 8] +(16) HashAggregate [codegen id : 4] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] @@ -178,40 +178,40 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Condition : isnotnull(sr_store_sk#16) (20) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#20] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] (23) ReusedExchange [Reuses operator id: 10] Output [1]: [s_store_sk#21] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_store_sk#16] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Keys [1]: [s_store_sk#21] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] @@ -222,10 +222,10 @@ Results [3]: [s_store_sk#21, sum#24, sum#25] Input [3]: [s_store_sk#21, sum#24, sum#25] Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#21, sum#24, sum#25] -(29) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 3] Input [3]: [s_store_sk#21, sum#24, sum#25] Keys [1]: [s_store_sk#21] Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] @@ -236,13 +236,13 @@ Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26 Input [3]: [s_store_sk#21, returns#28, profit_loss#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#7] Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 4] Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33] Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] @@ -253,23 +253,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 5] Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] (35) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#38] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#37] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 5] Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] -(38) HashAggregate [codegen id : 10] +(38) HashAggregate [codegen id : 5] Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Keys [1]: [cs_call_center_sk#34] Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] @@ -280,10 +280,10 @@ Results [3]: [cs_call_center_sk#34, sum#41, sum#42] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometColumnarToRow [codegen id : 11] +(40) CometNativeColumnarToRow Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 6] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Keys [1]: [cs_call_center_sk#34] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] @@ -301,23 +301,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] +(44) ColumnarToRow [codegen id : 7] Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] (45) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#50] -(46) BroadcastHashJoin [codegen id : 13] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cr_returned_date_sk#49] Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(47) Project [codegen id : 13] +(47) Project [codegen id : 7] Output [2]: [cr_return_amount#47, cr_net_loss#48] Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] -(48) HashAggregate [codegen id : 13] +(48) HashAggregate [codegen id : 7] Input [2]: [cr_return_amount#47, cr_net_loss#48] Keys: [] Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] @@ -328,7 +328,7 @@ Results [2]: [sum#53, sum#54] Input [2]: [sum#53, sum#54] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) CometColumnarToRow +(50) CometNativeColumnarToRow Input [2]: [sum#53, sum#54] (51) HashAggregate @@ -338,11 +338,11 @@ Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_ne Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] -(52) BroadcastNestedLoopJoin [codegen id : 14] +(52) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 8] Output [5]: [sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61] Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] @@ -354,23 +354,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 17] +(55) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -(56) Filter [codegen id : 17] +(56) Filter [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] Condition : isnotnull(ws_web_page_sk#62) (57) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#66] -(58) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#65] Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(59) Project [codegen id : 9] Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] @@ -385,24 +385,24 @@ ReadSchema: struct Input [1]: [wp_web_page_sk#67] Condition : isnotnull(wp_web_page_sk#67) -(62) CometColumnarToRow [codegen id : 16] +(62) CometNativeColumnarToRow Input [1]: [wp_web_page_sk#67] (63) BroadcastExchange Input [1]: [wp_web_page_sk#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(64) BroadcastHashJoin [codegen id : 17] +(64) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_web_page_sk#62] Right keys [1]: [wp_web_page_sk#67] Join type: Inner Join condition: None -(65) Project [codegen id : 17] +(65) Project [codegen id : 9] Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 9] Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Keys [1]: [wp_web_page_sk#67] Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] @@ -413,10 +413,10 @@ Results [3]: [wp_web_page_sk#67, sum#70, sum#71] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometColumnarToRow [codegen id : 22] +(68) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -(69) HashAggregate [codegen id : 22] +(69) HashAggregate [codegen id : 12] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Keys [1]: [wp_web_page_sk#67] Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] @@ -431,40 +431,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -(72) Filter [codegen id : 20] +(72) Filter [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] Condition : isnotnull(wr_web_page_sk#76) (73) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#80] -(74) BroadcastHashJoin [codegen id : 20] +(74) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_returned_date_sk#79] Right keys [1]: [d_date_sk#80] Join type: Inner Join condition: None -(75) Project [codegen id : 20] +(75) Project [codegen id : 10] Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] (76) ReusedExchange [Reuses operator id: 63] Output [1]: [wp_web_page_sk#81] -(77) BroadcastHashJoin [codegen id : 20] +(77) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_web_page_sk#76] Right keys [1]: [wp_web_page_sk#81] Join type: Inner Join condition: None -(78) Project [codegen id : 20] +(78) Project [codegen id : 10] Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -(79) HashAggregate [codegen id : 20] +(79) HashAggregate [codegen id : 10] Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Keys [1]: [wp_web_page_sk#81] Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] @@ -475,10 +475,10 @@ Results [3]: [wp_web_page_sk#81, sum#84, sum#85] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(81) CometColumnarToRow [codegen id : 21] +(81) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -(82) HashAggregate [codegen id : 21] +(82) HashAggregate [codegen id : 11] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Keys [1]: [wp_web_page_sk#81] Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] @@ -489,23 +489,23 @@ Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77) Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 22] +(84) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wp_web_page_sk#67] Right keys [1]: [wp_web_page_sk#81] Join type: LeftOuter Join condition: None -(85) Project [codegen id : 22] +(85) Project [codegen id : 12] Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93] Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] (86) Union -(87) Expand [codegen id : 23] +(87) Expand [codegen id : 13] Input [5]: [sales#14, returns#30, profit#31, channel#32, id#33] Arguments: [[sales#14, returns#30, profit#31, channel#32, id#33, 0], [sales#14, returns#30, profit#31, channel#32, null, 1], [sales#14, returns#30, profit#31, null, null, 3]], [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] -(88) HashAggregate [codegen id : 23] +(88) HashAggregate [codegen id : 13] Input [6]: [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] Keys [3]: [channel#94, id#95, spark_grouping_id#96] Functions [3]: [partial_sum(sales#14), partial_sum(returns#30), partial_sum(profit#31)] @@ -516,10 +516,10 @@ Results [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] Arguments: hashpartitioning(channel#94, id#95, spark_grouping_id#96, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(90) CometColumnarToRow [codegen id : 24] +(90) CometNativeColumnarToRow Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -(91) HashAggregate [codegen id : 24] +(91) HashAggregate [codegen id : 14] Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] Keys [3]: [channel#94, id#95, spark_grouping_id#96] Functions [3]: [sum(sales#14), sum(returns#30), sum(profit#31)] @@ -534,7 +534,7 @@ Arguments: 100, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (97) -+- * CometColumnarToRow (96) ++- CometNativeColumnarToRow (96) +- CometProject (95) +- CometFilter (94) +- CometNativeScan parquet spark_catalog.default.date_dim (93) @@ -555,7 +555,7 @@ Condition : (((isnotnull(d_date#115) AND (d_date#115 >= 2000-08-03)) AND (d_date Input [2]: [d_date_sk#6, d_date#115] Arguments: [d_date_sk#6], [d_date_sk#6] -(96) CometColumnarToRow [codegen id : 1] +(96) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (97) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt index d12b8dde24..5d2ca1a9d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -8,7 +8,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -20,22 +20,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -47,19 +47,19 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -68,12 +68,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -82,14 +82,14 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- BroadcastHashJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -101,17 +101,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -123,12 +123,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/simplified.txt index 4256e90759..5a732180f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/simplified.txt @@ -1,22 +1,22 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) + WholeStageCodegen (14) HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) + WholeStageCodegen (13) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (8) + WholeStageCodegen (4) Project [sales,returns,profit,profit_loss,s_store_sk] BroadcastHashJoin [s_store_sk,s_store_sk] HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_sk] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] Project [ss_ext_sales_price,ss_net_profit,s_store_sk] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -28,29 +28,25 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_sk] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] Project [sr_return_amt,sr_net_loss,s_store_sk] BroadcastHashJoin [sr_store_sk,s_store_sk] @@ -65,17 +61,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [sales,returns,profit,profit_loss,cs_call_center_sk] BroadcastNestedLoopJoin InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) + WholeStageCodegen (6) HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cs_call_center_sk] #8 - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] @@ -86,10 +82,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter ReusedExchange [d_date_sk] #3 HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #9 - WholeStageCodegen (13) + WholeStageCodegen (7) HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] Project [cr_return_amount,cr_net_loss] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] @@ -99,14 +95,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) + WholeStageCodegen (12) Project [sales,returns,profit,profit_loss,wp_web_page_sk] BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wp_web_page_sk] #10 - WholeStageCodegen (17) + WholeStageCodegen (9) HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] @@ -121,19 +117,17 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometNativeColumnarToRow + CometFilter [wp_web_page_sk] + CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter BroadcastExchange #12 - WholeStageCodegen (21) + WholeStageCodegen (11) HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [wp_web_page_sk] #13 - WholeStageCodegen (20) + WholeStageCodegen (10) HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] Project [wr_return_amt,wr_net_loss,wp_web_page_sk] BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/explain.txt index bb7ed0a50c..9155629835 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/explain.txt @@ -477,7 +477,7 @@ Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (90) -+- * CometColumnarToRow (89) ++- CometNativeColumnarToRow (89) +- CometProject (88) +- CometFilter (87) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) @@ -498,7 +498,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(89) CometColumnarToRow [codegen id : 1] +(89) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (90) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt index ed8a9e38ca..a16105c248 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/simplified.txt index a1243769e5..94722e1db2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/simplified.txt @@ -25,12 +25,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index bb7ed0a50c..9155629835 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -477,7 +477,7 @@ Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (90) -+- * CometColumnarToRow (89) ++- CometNativeColumnarToRow (89) +- CometProject (88) +- CometFilter (87) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) @@ -498,7 +498,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(89) CometColumnarToRow [codegen id : 1] +(89) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (90) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt index ed8a9e38ca..a16105c248 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a1243769e5..94722e1db2 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 @@ -25,12 +25,10 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/explain.txt index 4dc7ac380a..928636dfa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (70) +- * Project (69) - +- * CometColumnarToRow (68) + +- CometNativeColumnarToRow (68) +- CometSortMergeJoin (67) :- CometProject (45) : +- CometSortMergeJoin (44) @@ -372,7 +372,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner -(68) CometColumnarToRow [codegen id : 4] +(68) CometNativeColumnarToRow Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] (69) Project [codegen id : 4] @@ -387,7 +387,7 @@ Arguments: 100, [ratio#61 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -403,7 +403,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (74) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt index 65256f2124..af4b7000b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSortMergeJoin :- CometProject : +- CometSortMergeJoin @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/simplified.txt index ebd0840eec..5419c4229c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (4) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] @@ -24,11 +24,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/explain.txt index 3d3ea5ab86..e4118a724e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/simplified.txt index 77c05217f4..67056a3026 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/simplified.txt @@ -21,11 +21,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index 3d3ea5ab86..e4118a724e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort 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 77c05217f4..67056a3026 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 @@ -21,11 +21,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/explain.txt index 78ed64abe7..0cdeaf082a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (32) +- * Project (31) +- * BroadcastHashJoin Inner BuildRight (30) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -17,17 +17,17 @@ TakeOrderedAndProject (32) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (29) - +- * CometColumnarToRow (28) + +- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.customer (25) @@ -41,23 +41,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 37] Output [1]: [d_date_sk#10] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] @@ -76,20 +76,20 @@ Condition : (((isnotnull(s_number_employees#12) AND (s_number_employees#12 >= 20 Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] Arguments: [s_store_sk#11, s_city#13], [s_store_sk#11, s_city#13] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [s_store_sk#11, s_city#13] (11) BroadcastExchange Input [2]: [s_store_sk#11, s_city#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#4] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11, s_city#13] @@ -108,24 +108,24 @@ Condition : (((hd_dep_count#15 = 6) OR (hd_vehicle_count#16 > 2)) AND isnotnull( Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] Arguments: [hd_demo_sk#14], [hd_demo_sk#14] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#14] (18) BroadcastExchange Input [1]: [hd_demo_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13, hd_demo_sk#14] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] @@ -136,10 +136,10 @@ Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] @@ -161,20 +161,20 @@ Condition : isnotnull(c_customer_sk#25) Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] Arguments: [c_customer_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#26, 20, true, false, true) AS c_first_name#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#27, 30, true, false, true) AS c_last_name#29] -(28) CometColumnarToRow [codegen id : 5] +(28) CometNativeColumnarToRow Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] (29) BroadcastExchange Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [7]: [c_last_name#29, c_first_name#28, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#13] Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#28, c_last_name#29] @@ -186,7 +186,7 @@ Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (37) -+- * CometColumnarToRow (36) ++- CometNativeColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometNativeScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((isnotnull(d_dow#32) AND (d_dow#32 = 1)) AND d_year#31 IN (1999,20 Input [3]: [d_date_sk#10, d_year#31, d_dow#32] Arguments: [d_date_sk#10], [d_date_sk#10] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (37) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt index 05bd194c34..f24d730d8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject +- Project +- BroadcastHashJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -16,27 +16,27 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/simplified.txt index 1ee7a286a4..48b20af3a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/simplified.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (6) + WholeStageCodegen (2) Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] BroadcastHashJoin [ss_customer_sk,c_customer_sk] HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] @@ -20,35 +20,27 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + CometNativeColumnarToRow + CometProject [s_store_sk,s_city] + CometFilter [s_store_sk,s_number_employees,s_city] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/explain.txt index 2d2c6dc938..98be1506c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/explain.txt @@ -181,7 +181,7 @@ Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt index 012403275a..871cf47610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/simplified.txt index 5b252a906b..d9d0bedcfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/simplified.txt @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt index 2d2c6dc938..98be1506c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt @@ -181,7 +181,7 @@ Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) @@ -202,7 +202,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/extended.txt index 012403275a..871cf47610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 5b252a906b..d9d0bedcfb 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 @@ -17,12 +17,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dow] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/explain.txt index 754a8871d1..d7f41206bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (44) +- * HashAggregate (43) - +- * CometColumnarToRow (42) + +- CometNativeColumnarToRow (42) +- CometColumnarExchange (41) +- * HashAggregate (40) +- * Project (39) @@ -15,12 +15,12 @@ TakeOrderedAndProject (44) : : : +- Scan parquet spark_catalog.default.store_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.store (7) +- BroadcastExchange (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometHashAggregate (35) +- CometExchange (34) +- CometHashAggregate (33) @@ -53,23 +53,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 49] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -88,20 +88,20 @@ Condition : (isnotnull(s_store_sk#6) AND isnotnull(staticinvoke(class org.apache Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] Arguments: [s_store_sk#6, s_store_name#7, s_zip#9], [s_store_sk#6, s_store_name#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#8, 10, true, false, true) AS s_zip#9] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] (11) BroadcastExchange Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#9] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#9] @@ -207,24 +207,24 @@ Input [1]: [ca_zip#11] Keys [1]: [ca_zip#11] Functions: [] -(36) CometColumnarToRow [codegen id : 3] +(36) CometNativeColumnarToRow Input [1]: [ca_zip#11] (37) BroadcastExchange Input [1]: [ca_zip#11] Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] -(38) BroadcastHashJoin [codegen id : 4] +(38) BroadcastHashJoin [codegen id : 1] Left keys [1]: [substr(s_zip#9, 1, 2)] Right keys [1]: [substr(ca_zip#11, 1, 2)] Join type: Inner Join condition: None -(39) Project [codegen id : 4] +(39) Project [codegen id : 1] Output [2]: [ss_net_profit#2, s_store_name#7] Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#9, ca_zip#11] -(40) HashAggregate [codegen id : 4] +(40) HashAggregate [codegen id : 1] Input [2]: [ss_net_profit#2, s_store_name#7] Keys [1]: [s_store_name#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] @@ -235,10 +235,10 @@ Results [2]: [s_store_name#7, sum#21] Input [2]: [s_store_name#7, sum#21] Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(42) CometColumnarToRow [codegen id : 5] +(42) CometNativeColumnarToRow Input [2]: [s_store_name#7, sum#21] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 2] Input [2]: [s_store_name#7, sum#21] Keys [1]: [s_store_name#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] @@ -253,7 +253,7 @@ Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometNativeScan parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2) Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt index 257b7f0d77..8c9eb7b8c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -14,22 +14,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/simplified.txt index 4df6d8f659..97b1393e3b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [s_store_name,ss_net_profit] [sum,sum] Project [ss_net_profit,s_store_name] BroadcastHashJoin [s_zip,ca_zip] @@ -18,46 +18,40 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometNativeColumnarToRow + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter + CometNativeColumnarToRow + CometHashAggregate [ca_zip] + CometExchange [ca_zip] #5 CometHashAggregate [ca_zip] - CometExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #6 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #7 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #8 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] + CometBroadcastHashJoin [ca_zip,ca_zip] + CometProject [ca_zip] [ca_zip] + CometFilter [ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_zip] + CometBroadcastExchange [ca_zip] #6 + CometProject [ca_zip] + CometFilter [ca_zip,cnt] + CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] + CometExchange [ca_zip] #7 + CometHashAggregate [ca_zip,count] + CometProject [ca_zip] + CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] + CometProject [ca_zip] [ca_address_sk,ca_zip] + CometFilter [ca_address_sk,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + CometBroadcastExchange [c_current_addr_sk] #8 + CometProject [c_current_addr_sk] + CometFilter [c_current_addr_sk,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/explain.txt index 075f17ea3d..c84da45c11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/explain.txt @@ -253,7 +253,7 @@ Input [2]: [s_store_name#9, sum(ss_net_profit)#23] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt index 8e04bbeba7..4d7a7d1ce7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/simplified.txt index d1542ab607..fc8379fb0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/simplified.txt @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index 075f17ea3d..c84da45c11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -253,7 +253,7 @@ Input [2]: [s_store_name#9, sum(ss_net_profit)#23] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (49) -+- * CometColumnarToRow (48) ++- CometNativeColumnarToRow (48) +- CometProject (47) +- CometFilter (46) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) @@ -274,7 +274,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(48) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (49) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/extended.txt index 8e04bbeba7..4d7a7d1ce7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d1542ab607..fc8379fb0d 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 @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/explain.txt index f1e1a2ace1..5a669c7083 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (106) +CometNativeColumnarToRow (106) +- CometTakeOrderedAndProject (105) +- CometHashAggregate (104) +- CometExchange (103) @@ -581,14 +581,14 @@ Functions [3]: [sum(sales#74), sum(returns#75), sum(profit#76)] Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,id#90 ASC NULLS FIRST], output=[channel#89,id#90,sales#98,returns#99,profit#100]), [channel#89, id#90, sales#98, returns#99, profit#100], 100, 0, [channel#89 ASC NULLS FIRST, id#90 ASC NULLS FIRST], [channel#89, id#90, sales#98, returns#99, profit#100] -(106) CometColumnarToRow [codegen id : 4] +(106) CometNativeColumnarToRow Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (111) -+- * CometColumnarToRow (110) ++- CometNativeColumnarToRow (110) +- CometProject (109) +- CometFilter (108) +- CometNativeScan parquet spark_catalog.default.date_dim (107) @@ -609,7 +609,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(110) CometColumnarToRow [codegen id : 1] +(110) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (111) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt index 9d97e27318..6853afccad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/simplified.txt index 09e9224cef..3d30801890 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/simplified.txt @@ -1,124 +1,120 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - 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] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - 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] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #11 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - 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] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #15 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] + CometExchange [s_store_id] #2 + 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] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 + WholeStageCodegen (1) + Filter [ss_store_sk,ss_item_sk,ss_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #5 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #10 + 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] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #11 + WholeStageCodegen (2) + Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] + CometExchange [web_site_id] #14 + 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] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #15 + WholeStageCodegen (3) + Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt index f0a163bf69..ff987b999b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt @@ -576,7 +576,7 @@ Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -597,7 +597,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt index ae6fc19b60..dd2aed67bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt index 198646e0fd..9781eeab1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #5 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index f0a163bf69..ff987b999b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -576,7 +576,7 @@ Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -597,7 +597,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/extended.txt index ae6fc19b60..dd2aed67bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 198646e0fd..9781eeab1c 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 @@ -26,12 +26,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #5 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/explain.txt index 63856aaca5..c22b1ad83b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/explain.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject (54) : : +- * BroadcastHashJoin Inner BuildRight (38) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) @@ -20,18 +20,18 @@ TakeOrderedAndProject (54) : : : : : +- Scan parquet spark_catalog.default.catalog_returns (1) : : : : +- ReusedExchange (4) : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) + : : : +- CometNativeColumnarToRow (10) : : : +- CometProject (9) : : : +- CometFilter (8) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) : : +- BroadcastExchange (37) : : +- * Filter (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) + : : +- CometNativeColumnarToRow (30) : : +- CometColumnarExchange (29) : : +- * HashAggregate (28) : : +- * Project (27) @@ -44,12 +44,12 @@ TakeOrderedAndProject (54) : : : +- ReusedExchange (22) : : +- ReusedExchange (25) : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometProject (42) : +- CometFilter (41) : +- CometNativeScan parquet spark_catalog.default.customer (40) +- BroadcastExchange (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometNativeScan parquet spark_catalog.default.customer_address (47) @@ -63,23 +63,23 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] @@ -98,24 +98,24 @@ Condition : (isnotnull(ca_address_sk#7) AND isnotnull(staticinvoke(class org.apa Input [2]: [ca_address_sk#7, ca_state#8] Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#9] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [ca_address_sk#7, ca_state#9] (11) BroadcastExchange Input [2]: [ca_address_sk#7, ca_state#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#9] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] @@ -126,17 +126,17 @@ Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 11] +(16) CometNativeColumnarToRow Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -(17) HashAggregate [codegen id : 11] +(17) HashAggregate [codegen id : 5] Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] -(18) Filter [codegen id : 11] +(18) Filter [codegen id : 5] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) @@ -148,40 +148,40 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] +(20) ColumnarToRow [codegen id : 2] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -(21) Filter [codegen id : 6] +(21) Filter [codegen id : 2] Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] Condition : isnotnull(cr_returning_addr_sk#2) (22) ReusedExchange [Reuses operator id: 59] Output [1]: [d_date_sk#6] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returned_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(24) Project [codegen id : 6] +(24) Project [codegen id : 2] Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] (25) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#7, ca_state#9] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returning_addr_sk#2] Right keys [1]: [ca_address_sk#7] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 2] Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#9] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 2] Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] @@ -192,17 +192,17 @@ Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#17] Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#17] Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#17] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 3] Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#17] Keys [2]: [cr_returning_customer_sk#1, ca_state#9] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] Results [2]: [ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] -(32) HashAggregate [codegen id : 7] +(32) HashAggregate [codegen id : 3] Input [2]: [ctr_state#14, ctr_total_return#15] Keys [1]: [ctr_state#14] Functions [1]: [partial_avg(ctr_total_return#15)] @@ -213,17 +213,17 @@ Results [3]: [ctr_state#14, sum#20, count#21] Input [3]: [ctr_state#14, sum#20, count#21] Arguments: hashpartitioning(ctr_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 8] +(34) CometNativeColumnarToRow Input [3]: [ctr_state#14, sum#20, count#21] -(35) HashAggregate [codegen id : 8] +(35) HashAggregate [codegen id : 4] Input [3]: [ctr_state#14, sum#20, count#21] Keys [1]: [ctr_state#14] Functions [1]: [avg(ctr_total_return#15)] Aggregate Attributes [1]: [avg(ctr_total_return#15)#22] Results [2]: [(avg(ctr_total_return#15)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#14 AS ctr_state#14#24] -(36) Filter [codegen id : 8] +(36) Filter [codegen id : 4] Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) @@ -231,13 +231,13 @@ Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] -(38) BroadcastHashJoin [codegen id : 11] +(38) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_state#14] Right keys [1]: [ctr_state#14#24] Join type: Inner Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) -(39) Project [codegen id : 11] +(39) Project [codegen id : 5] Output [2]: [ctr_customer_sk#13, ctr_total_return#15] Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] @@ -256,20 +256,20 @@ Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] Arguments: [c_customer_sk#25, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#31, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#28, 10, true, false, true) AS c_salutation#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#34] -(43) CometColumnarToRow [codegen id : 9] +(43) CometNativeColumnarToRow Input [6]: [c_customer_sk#25, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34] (44) BroadcastExchange Input [6]: [c_customer_sk#25, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(45) BroadcastHashJoin [codegen id : 11] +(45) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ctr_customer_sk#13] Right keys [1]: [c_customer_sk#25] Join type: Inner Join condition: None -(46) Project [codegen id : 11] +(46) Project [codegen id : 5] Output [6]: [ctr_total_return#15, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34] Input [8]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#25, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34] @@ -288,20 +288,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [12]: [ca_address_sk#35, ca_street_number#36, ca_street_name#37, ca_street_type#38, ca_suite_number#39, ca_city#40, ca_county#41, ca_state#42, ca_zip#43, ca_country#44, ca_gmt_offset#45, ca_location_type#46] Arguments: [ca_address_sk#35, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52], [ca_address_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#36, 10, true, false, true) AS ca_street_number#47, ca_street_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_type#38, 15, true, false, true) AS ca_street_type#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_suite_number#39, 10, true, false, true) AS ca_suite_number#49, ca_city#40, ca_county#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#42, 2, true, false, true) AS ca_state#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#43, 10, true, false, true) AS ca_zip#51, ca_country#44, ca_gmt_offset#45, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_location_type#46, 20, true, false, true) AS ca_location_type#52] -(50) CometColumnarToRow [codegen id : 10] +(50) CometNativeColumnarToRow Input [12]: [ca_address_sk#35, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52] (51) BroadcastExchange Input [12]: [ca_address_sk#35, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#27] Right keys [1]: [ca_address_sk#35] Join type: Inner Join condition: None -(53) Project [codegen id : 11] +(53) Project [codegen id : 5] Output [16]: [c_customer_id#31, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52, ctr_total_return#15] Input [18]: [ctr_total_return#15, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34, ca_address_sk#35, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52] @@ -313,7 +313,7 @@ Arguments: 100, [c_customer_id#31 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIR Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (59) -+- * CometColumnarToRow (58) ++- CometNativeColumnarToRow (58) +- CometProject (57) +- CometFilter (56) +- CometNativeScan parquet spark_catalog.default.date_dim (55) @@ -334,7 +334,7 @@ Condition : ((isnotnull(d_year#53) AND (d_year#53 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#6, d_year#53] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(58) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (59) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt index 0e70ec42fa..841b2ca52d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt @@ -7,7 +7,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -19,28 +19,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange : : +- Filter : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -52,22 +52,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/simplified.txt index af74a164cc..378e1b8faf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - WholeStageCodegen (11) + WholeStageCodegen (5) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] @@ -8,10 +8,10 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] Filter [ctr_total_return] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] @@ -23,37 +23,33 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st Scan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #4 - WholeStageCodegen (8) + WholeStageCodegen (4) Filter [(avg(ctr_total_return) * 1.2)] HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] @@ -70,17 +66,13 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st ReusedExchange [ca_address_sk,ca_state] #3 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] InputAdapter BroadcastExchange #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometNativeColumnarToRow + CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/explain.txt index 8b66e63cf3..a7e6214731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/explain.txt @@ -285,7 +285,7 @@ Input [16]: [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt index 9708802e79..731fcea912 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/simplified.txt index a3a981dc54..d45005834b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index 8b66e63cf3..a7e6214731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -285,7 +285,7 @@ Input [16]: [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) @@ -306,7 +306,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/extended.txt index 9708802e79..731fcea912 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a3a981dc54..d45005834b 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/explain.txt index 93da88fcc1..12332a6d23 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (26) +CometNativeColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometColumnarExchange (23) @@ -11,7 +11,7 @@ : +- * BroadcastHashJoin Inner BuildRight (13) : :- * Project (11) : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -21,7 +21,7 @@ : : +- * ColumnarToRow (6) : : +- Scan parquet spark_catalog.default.inventory (5) : +- ReusedExchange (12) - +- * CometColumnarToRow (19) + +- CometNativeColumnarToRow (19) +- CometProject (18) +- CometFilter (17) +- CometNativeScan parquet spark_catalog.default.store_sales (16) @@ -42,7 +42,7 @@ Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) A Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] (5) Scan parquet spark_catalog.default.inventory @@ -68,26 +68,26 @@ Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] Input [2]: [inv_item_sk#7, inv_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [inv_item_sk#7] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] (12) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#11] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [inv_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] @@ -110,20 +110,20 @@ Condition : isnotnull(ss_item_sk#12) Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] Arguments: [ss_item_sk#12], [ss_item_sk#12] -(19) CometColumnarToRow +(19) CometNativeColumnarToRow Input [1]: [ss_item_sk#12] -(20) BroadcastHashJoin [codegen id : 4] +(20) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#12] Join type: Inner Join condition: None -(21) Project [codegen id : 4] +(21) Project [codegen id : 3] Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#12] -(22) HashAggregate [codegen id : 4] +(22) HashAggregate [codegen id : 3] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Functions: [] @@ -143,14 +143,14 @@ Functions: [] Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] -(26) CometColumnarToRow [codegen id : 5] +(26) CometNativeColumnarToRow Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.date_dim (27) @@ -171,7 +171,7 @@ Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#14] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt index c86e7ba2cb..8abf5c1fb1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -10,7 +10,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -21,16 +21,16 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/simplified.txt index b60bf92c5b..b8b1c8b053 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/simplified.txt @@ -1,45 +1,41 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + CometHashAggregate [i_item_id,i_item_desc,i_current_price] + CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,ss_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Project [inv_item_sk,inv_date_sk] + Filter [inv_quantity_on_hand,inv_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + CometNativeColumnarToRow + CometProject [ss_item_sk] + CometFilter [ss_item_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/explain.txt index 698608e16d..9380337fed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt index 9fbb7893b8..2ab6dd2d0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/simplified.txt index 28d160a4a8..b92f067e53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index 698608e16d..9380337fed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -152,7 +152,7 @@ Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/extended.txt index 9fbb7893b8..2ab6dd2d0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt index 28d160a4a8..b92f067e53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/explain.txt index 8af6504881..3b2e2d4338 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/explain.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject (50) :- * Project (33) : +- * BroadcastHashJoin Inner BuildRight (32) : :- * HashAggregate (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometColumnarExchange (15) : : +- * HashAggregate (14) : : +- * Project (13) @@ -16,14 +16,14 @@ TakeOrderedAndProject (50) : : : : +- * ColumnarToRow (2) : : : : +- Scan parquet spark_catalog.default.store_returns (1) : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) + : : : +- CometNativeColumnarToRow (7) : : : +- CometProject (6) : : : +- CometFilter (5) : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : +- ReusedExchange (11) : +- BroadcastExchange (31) : +- * HashAggregate (30) - : +- * CometColumnarToRow (29) + : +- CometNativeColumnarToRow (29) : +- CometColumnarExchange (28) : +- * HashAggregate (27) : +- * Project (26) @@ -37,7 +37,7 @@ TakeOrderedAndProject (50) : +- ReusedExchange (24) +- BroadcastExchange (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * Project (42) @@ -59,10 +59,10 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr PushedFilters: [IsNotNull(sr_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] Condition : isnotnull(sr_item_sk#1) @@ -81,37 +81,37 @@ Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache. Input [2]: [i_item_sk#5, i_item_id#6] Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [i_item_sk#5, i_item_id#7] (8) BroadcastExchange Input [2]: [i_item_sk#5, i_item_id#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [sr_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] (11) ReusedExchange [Reuses operator id: 64] Output [1]: [d_date_sk#8] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [sr_returned_date_sk#3] Right keys [1]: [d_date_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [2]: [sr_return_quantity#2, i_item_id#7] Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [2]: [sr_return_quantity#2, i_item_id#7] Keys [1]: [i_item_id#7] Functions [1]: [partial_sum(sr_return_quantity#2)] @@ -122,10 +122,10 @@ Results [2]: [i_item_id#7, sum#10] Input [2]: [i_item_id#7, sum#10] Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 12] +(16) CometNativeColumnarToRow Input [2]: [i_item_id#7, sum#10] -(17) HashAggregate [codegen id : 12] +(17) HashAggregate [codegen id : 6] Input [2]: [i_item_id#7, sum#10] Keys [1]: [i_item_id#7] Functions [1]: [sum(sr_return_quantity#2)] @@ -140,40 +140,40 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#16), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 2] Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -(20) Filter [codegen id : 6] +(20) Filter [codegen id : 2] Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] Condition : isnotnull(cr_item_sk#14) (21) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#18, i_item_id#19] -(22) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_item_sk#14] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(23) Project [codegen id : 2] Output [3]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19] Input [5]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16, i_item_sk#18, i_item_id#19] (24) ReusedExchange [Reuses operator id: 75] Output [1]: [d_date_sk#20] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returned_date_sk#16] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(26) Project [codegen id : 2] Output [2]: [cr_return_quantity#15, i_item_id#19] Input [4]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19, d_date_sk#20] -(27) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 2] Input [2]: [cr_return_quantity#15, i_item_id#19] Keys [1]: [i_item_id#19] Functions [1]: [partial_sum(cr_return_quantity#15)] @@ -184,10 +184,10 @@ Results [2]: [i_item_id#19, sum#22] Input [2]: [i_item_id#19, sum#22] Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(29) CometColumnarToRow [codegen id : 7] +(29) CometNativeColumnarToRow Input [2]: [i_item_id#19, sum#22] -(30) HashAggregate [codegen id : 7] +(30) HashAggregate [codegen id : 3] Input [2]: [i_item_id#19, sum#22] Keys [1]: [i_item_id#19] Functions [1]: [sum(cr_return_quantity#15)] @@ -198,13 +198,13 @@ Results [2]: [i_item_id#19 AS item_id#24, sum(cr_return_quantity#15)#23 AS cr_it Input [2]: [item_id#24, cr_item_qty#25] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 12] +(32) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#24] Join type: Inner Join condition: None -(33) Project [codegen id : 12] +(33) Project [codegen id : 6] Output [3]: [item_id#12, sr_item_qty#13, cr_item_qty#25] Input [4]: [item_id#12, sr_item_qty#13, item_id#24, cr_item_qty#25] @@ -216,40 +216,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#28), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] +(35) ColumnarToRow [codegen id : 4] Input [3]: [wr_item_sk#26, wr_return_quantity#27, wr_returned_date_sk#28] -(36) Filter [codegen id : 10] +(36) Filter [codegen id : 4] Input [3]: [wr_item_sk#26, wr_return_quantity#27, wr_returned_date_sk#28] Condition : isnotnull(wr_item_sk#26) (37) ReusedExchange [Reuses operator id: 8] Output [2]: [i_item_sk#29, i_item_id#30] -(38) BroadcastHashJoin [codegen id : 10] +(38) BroadcastHashJoin [codegen id : 4] Left keys [1]: [wr_item_sk#26] Right keys [1]: [i_item_sk#29] Join type: Inner Join condition: None -(39) Project [codegen id : 10] +(39) Project [codegen id : 4] Output [3]: [wr_return_quantity#27, wr_returned_date_sk#28, i_item_id#30] Input [5]: [wr_item_sk#26, wr_return_quantity#27, wr_returned_date_sk#28, i_item_sk#29, i_item_id#30] (40) ReusedExchange [Reuses operator id: 75] Output [1]: [d_date_sk#31] -(41) BroadcastHashJoin [codegen id : 10] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [wr_returned_date_sk#28] Right keys [1]: [d_date_sk#31] Join type: Inner Join condition: None -(42) Project [codegen id : 10] +(42) Project [codegen id : 4] Output [2]: [wr_return_quantity#27, i_item_id#30] Input [4]: [wr_return_quantity#27, wr_returned_date_sk#28, i_item_id#30, d_date_sk#31] -(43) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 4] Input [2]: [wr_return_quantity#27, i_item_id#30] Keys [1]: [i_item_id#30] Functions [1]: [partial_sum(wr_return_quantity#27)] @@ -260,10 +260,10 @@ Results [2]: [i_item_id#30, sum#33] Input [2]: [i_item_id#30, sum#33] Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 11] +(45) CometNativeColumnarToRow Input [2]: [i_item_id#30, sum#33] -(46) HashAggregate [codegen id : 11] +(46) HashAggregate [codegen id : 5] Input [2]: [i_item_id#30, sum#33] Keys [1]: [i_item_id#30] Functions [1]: [sum(wr_return_quantity#27)] @@ -274,13 +274,13 @@ Results [2]: [i_item_id#30 AS item_id#35, sum(wr_return_quantity#27)#34 AS wr_it Input [2]: [item_id#35, wr_item_qty#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] -(48) BroadcastHashJoin [codegen id : 12] +(48) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_id#12] Right keys [1]: [item_id#35] Join type: Inner Join condition: None -(49) Project [codegen id : 12] +(49) Project [codegen id : 6] Output [8]: [item_id#12, sr_item_qty#13, (((cast(sr_item_qty#13 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#25) + wr_item_qty#36) as double)))) / 3.0) * 100.0) AS sr_dev#37, cr_item_qty#25, (((cast(cr_item_qty#25 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#25) + wr_item_qty#36) as double)))) / 3.0) * 100.0) AS cr_dev#38, wr_item_qty#36, (((cast(wr_item_qty#36 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#25) + wr_item_qty#36) as double)))) / 3.0) * 100.0) AS wr_dev#39, (cast(((sr_item_qty#13 + cr_item_qty#25) + wr_item_qty#36) as decimal(20,0)) / 3.0) AS average#40] Input [5]: [item_id#12, sr_item_qty#13, cr_item_qty#25, item_id#35, wr_item_qty#36] @@ -292,7 +292,7 @@ Arguments: 100, [item_id#12 ASC NULLS FIRST, sr_item_qty#13 ASC NULLS FIRST], [i Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (64) -+- * CometColumnarToRow (63) ++- CometNativeColumnarToRow (63) +- CometProject (62) +- CometBroadcastHashJoin (61) :- CometFilter (52) @@ -364,7 +364,7 @@ Arguments: [d_date#41], [d_date#41#44], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#41] Arguments: [d_date_sk#8], [d_date_sk#8] -(63) CometColumnarToRow [codegen id : 1] +(63) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (64) BroadcastExchange @@ -373,7 +373,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#16 IN dynamicpruning#17 BroadcastExchange (75) -+- * CometColumnarToRow (74) ++- CometNativeColumnarToRow (74) +- CometProject (73) +- CometBroadcastHashJoin (72) :- CometFilter (66) @@ -427,7 +427,7 @@ Arguments: [d_date#45], [d_date#41], LeftSemi, BuildRight Input [2]: [d_date_sk#20, d_date#45] Arguments: [d_date_sk#20], [d_date_sk#20] -(74) CometColumnarToRow [codegen id : 1] +(74) CometNativeColumnarToRow Input [1]: [d_date_sk#20] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt index e0024369f8..eac3126bae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -30,12 +30,12 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter @@ -50,7 +50,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -62,7 +62,7 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -76,12 +76,12 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter @@ -96,7 +96,7 @@ TakeOrderedAndProject : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -108,12 +108,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/simplified.txt index 1e0e8fea70..fcecf69443 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - WholeStageCodegen (12) + WholeStageCodegen (6) Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] BroadcastHashJoin [item_id,item_id] Project [item_id,sr_item_qty,cr_item_qty] BroadcastHashJoin [item_id,item_id] HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [i_item_id,sr_return_quantity] [sum,sum] Project [sr_return_quantity,i_item_id] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -20,39 +20,35 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] [d_week_seq] + CometFilter [d_date,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) + WholeStageCodegen (3) HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,cr_return_quantity] [sum,sum] Project [cr_return_quantity,i_item_id] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] @@ -64,30 +60,28 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #9 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - ReusedExchange [d_week_seq] #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #9 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + ReusedExchange [d_week_seq] #4 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter ReusedExchange [d_date_sk] #8 InputAdapter BroadcastExchange #10 - WholeStageCodegen (11) + WholeStageCodegen (5) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #11 - WholeStageCodegen (10) + WholeStageCodegen (4) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] BroadcastHashJoin [wr_returned_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/explain.txt index a700acceb7..582dce16d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/explain.txt @@ -306,7 +306,7 @@ Input [8]: [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (68) -+- * CometColumnarToRow (67) ++- CometNativeColumnarToRow (67) +- CometProject (66) +- CometBroadcastHashJoin (65) :- CometFilter (56) @@ -378,7 +378,7 @@ Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (68) BroadcastExchange @@ -387,7 +387,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#15 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometBroadcastHashJoin (76) :- CometFilter (70) @@ -441,7 +441,7 @@ Arguments: [d_date#19], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#18, d_date#19] Arguments: [d_date_sk#18], [d_date_sk#18] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [1]: [d_date_sk#18] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt index d325825292..9ab4ae91f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -57,7 +57,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/simplified.txt index bff9c86587..b463ea2999 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/simplified.txt @@ -17,21 +17,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #5 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] @@ -54,18 +52,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #10 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - ReusedExchange [d_week_seq] #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #10 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + ReusedExchange [d_week_seq] #4 ReusedExchange [i_item_sk,i_item_id] #5 CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index a700acceb7..582dce16d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -306,7 +306,7 @@ Input [8]: [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 BroadcastExchange (68) -+- * CometColumnarToRow (67) ++- CometNativeColumnarToRow (67) +- CometProject (66) +- CometBroadcastHashJoin (65) :- CometFilter (56) @@ -378,7 +378,7 @@ Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(67) CometColumnarToRow [codegen id : 1] +(67) CometNativeColumnarToRow Input [1]: [d_date_sk#8] (68) BroadcastExchange @@ -387,7 +387,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#15 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometBroadcastHashJoin (76) :- CometFilter (70) @@ -441,7 +441,7 @@ Arguments: [d_date#19], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#18, d_date#19] Arguments: [d_date_sk#18], [d_date_sk#18] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [1]: [d_date_sk#18] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/extended.txt index d325825292..9ab4ae91f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -57,7 +57,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter 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 bff9c86587..b463ea2999 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 @@ -17,21 +17,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #3 + CometProject [d_date] [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] [d_week_seq] + CometFilter [d_date,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] CometBroadcastExchange [i_item_sk,i_item_id] #5 CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] @@ -54,18 +52,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #10 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - ReusedExchange [d_week_seq] #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometBroadcastHashJoin [d_date_sk,d_date,d_date] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [d_date] #10 + CometProject [d_date] + CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] + ReusedExchange [d_week_seq] #4 ReusedExchange [i_item_sk,i_item_id] #5 CometBroadcastExchange [d_date_sk] #11 CometProject [d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/explain.txt index 63140cfc2d..98e5328b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (33) +CometNativeColumnarToRow (33) +- CometTakeOrderedAndProject (32) +- CometProject (31) +- CometBroadcastHashJoin (30) @@ -185,6 +185,6 @@ Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 Input [3]: [customer_id#20, customername#21, c_customer_id#7] Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [customer_id#20, customername#21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/extended.txt index b025c868d1..402322f69f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/simplified.txt index 9df82b0e98..73465fcf59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/simplified.txt @@ -1,35 +1,33 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] - CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 - CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] - CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] - CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] - CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] - CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastExchange [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] +CometNativeColumnarToRow + CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] + CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] + CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk,ib_income_band_sk] + CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] + CometBroadcastHashJoin [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk,hd_demo_sk,hd_income_band_sk] + CometProject [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + CometProject [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] + CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastExchange [ca_address_sk] #2 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_city] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [cd_demo_sk] #3 + CometFilter [cd_demo_sk] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + CometBroadcastExchange [ib_income_band_sk] #5 + CometProject [ib_income_band_sk] + CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + CometProject [sr_cdemo_sk] + CometFilter [sr_cdemo_sk,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/explain.txt index 204cd4ea7a..0f21e58ceb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (52) +- * HashAggregate (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- CometColumnarExchange (49) +- * HashAggregate (48) +- * Project (47) @@ -22,32 +22,32 @@ TakeOrderedAndProject (52) : : : : : : : +- * Filter (3) : : : : : : : +- * ColumnarToRow (2) : : : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : : : +- * CometColumnarToRow (8) + : : : : : : +- CometNativeColumnarToRow (8) : : : : : : +- CometProject (7) : : : : : : +- CometFilter (6) : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (5) : : : : : +- BroadcastExchange (14) - : : : : : +- * CometColumnarToRow (13) + : : : : : +- CometNativeColumnarToRow (13) : : : : : +- CometFilter (12) : : : : : +- CometNativeScan parquet spark_catalog.default.web_page (11) : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) + : : : : +- CometNativeColumnarToRow (20) : : : : +- CometProject (19) : : : : +- CometFilter (18) : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (17) : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) + : : : +- CometNativeColumnarToRow (27) : : : +- CometProject (26) : : : +- CometFilter (25) : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (24) : : +- BroadcastExchange (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometProject (33) : : +- CometFilter (32) : : +- CometNativeScan parquet spark_catalog.default.customer_address (31) : +- ReusedExchange (38) +- BroadcastExchange (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.reason (41) @@ -87,16 +87,16 @@ Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -(8) CometColumnarToRow +(8) CometNativeColumnarToRow Input [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -(9) BroadcastHashJoin [codegen id : 8] +(9) BroadcastHashJoin [codegen id : 2] Left keys [2]: [ws_item_sk#1, ws_order_number#3] Right keys [2]: [wr_item_sk#9, wr_order_number#14] Join type: Inner Join condition: None -(10) Project [codegen id : 8] +(10) Project [codegen id : 2] Output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] @@ -111,20 +111,20 @@ ReadSchema: struct Input [1]: [wp_web_page_sk#18] Condition : isnotnull(wp_web_page_sk#18) -(13) CometColumnarToRow [codegen id : 2] +(13) CometNativeColumnarToRow Input [1]: [wp_web_page_sk#18] (14) BroadcastExchange Input [1]: [wp_web_page_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(15) BroadcastHashJoin [codegen id : 8] +(15) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_web_page_sk#2] Right keys [1]: [wp_web_page_sk#18] Join type: Inner Join condition: None -(16) Project [codegen id : 8] +(16) Project [codegen id : 2] Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] @@ -143,20 +143,20 @@ Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(staticinvoke(class org.apa Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] (21) BroadcastExchange Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 8] +(22) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_refunded_cdemo_sk#10] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) -(23) Project [codegen id : 8] +(23) Project [codegen id : 2] Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] @@ -175,20 +175,20 @@ Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(staticinvoke(class org.apac Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true) AS cd_education_status#28] -(27) CometColumnarToRow [codegen id : 4] +(27) CometNativeColumnarToRow Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] (28) BroadcastExchange Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, string, true], input[2, string, true]),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 8] +(29) BroadcastHashJoin [codegen id : 2] Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23] Right keys [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] Join type: Inner Join condition: None -(30) Project [codegen id : 8] +(30) Project [codegen id : 2] Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] @@ -207,33 +207,33 @@ Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) AS ca_state#32] -(34) CometColumnarToRow [codegen id : 5] +(34) CometNativeColumnarToRow Input [2]: [ca_address_sk#29, ca_state#32] (35) BroadcastExchange Input [2]: [ca_address_sk#29, ca_state#32] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_refunded_addr_sk#11] Right keys [1]: [ca_address_sk#29] Join type: Inner Join condition: ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) -(37) Project [codegen id : 8] +(37) Project [codegen id : 2] Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] (38) ReusedExchange [Reuses operator id: 57] Output [1]: [d_date_sk#33] -(39) BroadcastHashJoin [codegen id : 8] +(39) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#7] Right keys [1]: [d_date_sk#33] Join type: Inner Join condition: None -(40) Project [codegen id : 8] +(40) Project [codegen id : 2] Output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] @@ -252,24 +252,24 @@ Condition : isnotnull(r_reason_sk#34) Input [2]: [r_reason_sk#34, r_reason_desc#35] Arguments: [r_reason_sk#34, r_reason_desc#36], [r_reason_sk#34, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#35, 100, true, false, true) AS r_reason_desc#36] -(44) CometColumnarToRow [codegen id : 7] +(44) CometNativeColumnarToRow Input [2]: [r_reason_sk#34, r_reason_desc#36] (45) BroadcastExchange Input [2]: [r_reason_sk#34, r_reason_desc#36] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 8] +(46) BroadcastHashJoin [codegen id : 2] Left keys [1]: [wr_reason_sk#13] Right keys [1]: [r_reason_sk#34] Join type: Inner Join condition: None -(47) Project [codegen id : 8] +(47) Project [codegen id : 2] Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#34, r_reason_desc#36] -(48) HashAggregate [codegen id : 8] +(48) HashAggregate [codegen id : 2] Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] Keys [1]: [r_reason_desc#36] Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] @@ -280,10 +280,10 @@ Results [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, coun Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] Arguments: hashpartitioning(r_reason_desc#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) CometColumnarToRow [codegen id : 9] +(50) CometNativeColumnarToRow Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] -(51) HashAggregate [codegen id : 9] +(51) HashAggregate [codegen id : 3] Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] Keys [1]: [r_reason_desc#36] Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] @@ -298,7 +298,7 @@ Arguments: 100, [substr(r_reason_desc, 1, 20)#52 ASC NULLS FIRST, avg(ws_quantit Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#33, d_year#56] Arguments: [d_date_sk#33], [d_date_sk#33] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt index 7f8b90a86c..aed7b365be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -23,40 +23,40 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_page : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/simplified.txt index 68d46e608d..e16d15b6c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (9) + WholeStageCodegen (3) HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [r_reason_desc] #1 - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] BroadcastHashJoin [wr_reason_sk,r_reason_sk] @@ -29,55 +29,43 @@ TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refu Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] + CometNativeColumnarToRow + CometFilter [wp_web_page_sk] + CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter + CometNativeColumnarToRow CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #7 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] + CometNativeColumnarToRow + CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] + CometFilter [r_reason_sk,r_reason_desc] + CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/explain.txt index c263f03d76..25fbead789 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/explain.txt @@ -278,7 +278,7 @@ Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunde Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -299,7 +299,7 @@ Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#33, d_year#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt index e2383394d1..5c782b95f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/simplified.txt index dfcca32f03..1e7b26f922 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/simplified.txt @@ -24,12 +24,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index c263f03d76..25fbead789 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -278,7 +278,7 @@ Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunde Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -299,7 +299,7 @@ Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#33, d_year#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#33] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/extended.txt index e2383394d1..5c782b95f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 dfcca32f03..1e7b26f922 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 @@ -24,12 +24,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] CometFilter [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/explain.txt index 3f745e99c5..1aa03c12e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (24) +- * Project (23) +- Window (22) - +- * CometColumnarToRow (21) + +- CometNativeColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometColumnarExchange (16) +- * HashAggregate (15) +- * Expand (14) @@ -19,7 +19,7 @@ TakeOrderedAndProject (24) : : +- Scan parquet spark_catalog.default.web_sales (1) : +- ReusedExchange (4) +- BroadcastExchange (11) - +- * CometColumnarToRow (10) + +- CometNativeColumnarToRow (10) +- CometProject (9) +- CometFilter (8) +- CometNativeScan parquet spark_catalog.default.item (7) @@ -33,23 +33,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ws_item_sk#1, ws_net_paid#2] Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] @@ -68,28 +68,28 @@ Condition : isnotnull(i_item_sk#6) Input [3]: [i_item_sk#6, i_class#7, i_category#8] Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#6, i_class#9, i_category#10] (11) BroadcastExchange Input [3]: [i_item_sk#6, i_class#9, i_category#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [ws_net_paid#2, i_category#10, i_class#9] Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] -(14) Expand [codegen id : 3] +(14) Expand [codegen id : 1] Input [3]: [ws_net_paid#2, i_category#10, i_class#9] Arguments: [[ws_net_paid#2, i_category#10, i_class#9, 0], [ws_net_paid#2, i_category#10, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 1] Input [4]: [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] @@ -100,10 +100,10 @@ Results [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] Arguments: hashpartitioning(i_category#11, i_class#12, spark_grouping_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 2] Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] @@ -118,14 +118,14 @@ Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(21) CometColumnarToRow [codegen id : 5] +(21) CometNativeColumnarToRow Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] (22) Window Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] -(23) Project [codegen id : 6] +(23) Project [codegen id : 3] Output [5]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] Input [8]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] @@ -137,7 +137,7 @@ Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ Condition : (((isnotnull(d_month_seq#23) AND (d_month_seq#23 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#23] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt index 41081debd9..a60a3158a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -18,17 +18,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/simplified.txt index 0b24fe234a..be50cc4b5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/simplified.txt @@ -1,44 +1,38 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] + Expand [ws_net_paid,i_category,i_class] + Project [ws_net_paid,i_category,i_class] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt index 1af5e2612c..c899fb891f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt @@ -138,7 +138,7 @@ Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -159,7 +159,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt index dfc810b108..6444b727e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt index 29e2d72920..d1e99b5c9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index 1af5e2612c..c899fb891f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -138,7 +138,7 @@ Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) @@ -159,7 +159,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt index dfc810b108..6444b727e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 29e2d72920..d1e99b5c9b 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 @@ -20,12 +20,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/explain.txt index b3086b072a..d6dacf8406 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == * HashAggregate (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometColumnarExchange (50) +- * HashAggregate (49) +- * Project (48) +- * BroadcastHashJoin LeftAnti BuildRight (47) :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (17) + : :- CometNativeColumnarToRow (17) : : +- CometHashAggregate (16) : : +- CometColumnarExchange (15) : : +- * HashAggregate (14) @@ -19,12 +19,12 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.customer (7) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometHashAggregate (29) : +- CometColumnarExchange (28) : +- * HashAggregate (27) @@ -38,7 +38,7 @@ : : +- ReusedExchange (21) : +- ReusedExchange (24) +- BroadcastExchange (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometHashAggregate (44) +- CometColumnarExchange (43) +- * HashAggregate (42) @@ -61,23 +61,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] Condition : isnotnull(ss_customer_sk#1) (4) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#4, d_date#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#2] Right keys [1]: [d_date_sk#4] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, d_date#5] Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] @@ -96,24 +96,24 @@ Condition : isnotnull(c_customer_sk#6) Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#7, 20, true, false, true) AS c_first_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#8, 30, true, false, true) AS c_last_name#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] (11) BroadcastExchange Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [c_last_name#10, c_first_name#9, d_date#5] Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [c_last_name#10, c_first_name#9, d_date#5] Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] Functions: [] @@ -129,7 +129,7 @@ Input [3]: [c_last_name#10, c_first_name#9, d_date#5] Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] Functions: [] -(17) CometColumnarToRow [codegen id : 12] +(17) CometNativeColumnarToRow Input [3]: [c_last_name#10, c_first_name#9, d_date#5] (18) Scan parquet spark_catalog.default.catalog_sales @@ -140,40 +140,40 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -(20) Filter [codegen id : 6] +(20) Filter [codegen id : 2] Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] Condition : isnotnull(cs_bill_customer_sk#11) (21) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#13, d_date#14] -(22) BroadcastHashJoin [codegen id : 6] +(22) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(23) Project [codegen id : 6] +(23) Project [codegen id : 2] Output [2]: [cs_bill_customer_sk#11, d_date#14] Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] (24) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] -(25) BroadcastHashJoin [codegen id : 6] +(25) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_bill_customer_sk#11] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(26) Project [codegen id : 6] +(26) Project [codegen id : 2] Output [3]: [c_last_name#17, c_first_name#16, d_date#14] Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] -(27) HashAggregate [codegen id : 6] +(27) HashAggregate [codegen id : 2] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] @@ -189,14 +189,14 @@ Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] Functions: [] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (31) BroadcastExchange Input [3]: [c_last_name#17, c_first_name#16, d_date#14] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 12] +(32) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] Join type: LeftAnti @@ -210,40 +210,40 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -(35) Filter [codegen id : 10] +(35) Filter [codegen id : 3] Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] Condition : isnotnull(ws_bill_customer_sk#18) (36) ReusedExchange [Reuses operator id: 57] Output [2]: [d_date_sk#20, d_date#21] -(37) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(38) Project [codegen id : 3] Output [2]: [ws_bill_customer_sk#18, d_date#21] Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] (39) ReusedExchange [Reuses operator id: 11] Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] -(40) BroadcastHashJoin [codegen id : 10] +(40) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_bill_customer_sk#18] Right keys [1]: [c_customer_sk#22] Join type: Inner Join condition: None -(41) Project [codegen id : 10] +(41) Project [codegen id : 3] Output [3]: [c_last_name#24, c_first_name#23, d_date#21] Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 3] Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] @@ -259,24 +259,24 @@ Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] Functions: [] -(45) CometColumnarToRow [codegen id : 11] +(45) CometNativeColumnarToRow Input [3]: [c_last_name#24, c_first_name#23, d_date#21] (46) BroadcastExchange Input [3]: [c_last_name#24, c_first_name#23, d_date#21] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] -(47) BroadcastHashJoin [codegen id : 12] +(47) BroadcastHashJoin [codegen id : 4] Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] Join type: LeftAnti Join condition: None -(48) Project [codegen id : 12] +(48) Project [codegen id : 4] Output: [] Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -(49) HashAggregate [codegen id : 12] +(49) HashAggregate [codegen id : 4] Input: [] Keys: [] Functions [1]: [partial_count(1)] @@ -287,10 +287,10 @@ Results [1]: [count#26] Input [1]: [count#26] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) CometColumnarToRow [codegen id : 13] +(51) CometNativeColumnarToRow Input [1]: [count#26] -(52) HashAggregate [codegen id : 13] +(52) HashAggregate [codegen id : 5] Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] @@ -301,7 +301,7 @@ Results [1]: [count(1)#27 AS count(1)#28] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (57) -+- * CometColumnarToRow (56) ++- CometNativeColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometNativeScan parquet spark_catalog.default.date_dim (53) @@ -322,7 +322,7 @@ Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_mo Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(56) CometColumnarToRow [codegen id : 1] +(56) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (57) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt index 638b8865e0..7a9e62d57c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt @@ -1,11 +1,11 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -18,22 +18,22 @@ HashAggregate : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometColumnarExchange : +- HashAggregate @@ -46,17 +46,17 @@ HashAggregate : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate @@ -69,12 +69,12 @@ HashAggregate : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/simplified.txt index afc302e779..25e4104be0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ -WholeStageCodegen (13) +WholeStageCodegen (5) HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (12) + WholeStageCodegen (4) HashAggregate [count,count] Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [c_last_name,c_first_name,d_date] CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,d_date] Project [c_last_name,c_first_name,d_date] BroadcastHashJoin [ss_customer_sk,c_customer_sk] @@ -24,63 +24,55 @@ WholeStageCodegen (13) Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] InputAdapter ReusedExchange [d_date_sk,d_date] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometNativeColumnarToRow + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometNativeColumnarToRow + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + CometNativeColumnarToRow + CometHashAggregate [c_last_name,c_first_name,d_date] + CometColumnarExchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_bill_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_date] #3 + Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/explain.txt index fc6b12b834..0cfb76e0f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/explain.txt @@ -292,7 +292,7 @@ Results [1]: [count(1)#30 AS count(1)#31] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (56) -+- * CometColumnarToRow (55) ++- CometNativeColumnarToRow (55) +- CometProject (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -313,7 +313,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(55) CometColumnarToRow [codegen id : 1] +(55) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (56) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt index ddee139acf..0c51ae09a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ HashAggregate : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/simplified.txt index f687139735..e1555c4783 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index fc6b12b834..0cfb76e0f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -292,7 +292,7 @@ Results [1]: [count(1)#30 AS count(1)#31] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 BroadcastExchange (56) -+- * CometColumnarToRow (55) ++- CometNativeColumnarToRow (55) +- CometProject (54) +- CometFilter (53) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) @@ -313,7 +313,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(55) CometColumnarToRow [codegen id : 1] +(55) CometNativeColumnarToRow Input [2]: [d_date_sk#4, d_date#5] (56) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/extended.txt index ddee139acf..0c51ae09a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/extended.txt @@ -17,7 +17,7 @@ HashAggregate : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt index f687139735..e1555c4783 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt @@ -21,12 +21,10 @@ WholeStageCodegen (4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #4 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/explain.txt index 25b4c305f2..d2d4ce18d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/explain.txt @@ -6,7 +6,7 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) +: : : : : : :- CometNativeColumnarToRow (25) : : : : : : : +- CometHashAggregate (24) : : : : : : : +- CometExchange (23) : : : : : : : +- CometHashAggregate (22) @@ -32,7 +32,7 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) +: : : : : : +- CometNativeColumnarToRow (44) : : : : : : +- CometHashAggregate (43) : : : : : : +- CometExchange (42) : : : : : : +- CometHashAggregate (41) @@ -52,7 +52,7 @@ : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) +: : : : : +- CometNativeColumnarToRow (65) : : : : : +- CometHashAggregate (64) : : : : : +- CometExchange (63) : : : : : +- CometHashAggregate (62) @@ -72,7 +72,7 @@ : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) +: : : : +- CometNativeColumnarToRow (86) : : : : +- CometHashAggregate (85) : : : : +- CometExchange (84) : : : : +- CometHashAggregate (83) @@ -92,7 +92,7 @@ : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) +: : : +- CometNativeColumnarToRow (107) : : : +- CometHashAggregate (106) : : : +- CometExchange (105) : : : +- CometHashAggregate (104) @@ -112,7 +112,7 @@ : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) +: : +- CometNativeColumnarToRow (128) : : +- CometHashAggregate (127) : : +- CometExchange (126) : : +- CometHashAggregate (125) @@ -132,7 +132,7 @@ : : : +- CometNativeScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) +: +- CometNativeColumnarToRow (149) : +- CometHashAggregate (148) : +- CometExchange (147) : +- CometHashAggregate (146) @@ -152,7 +152,7 @@ : : +- CometNativeScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * CometColumnarToRow (170) + +- CometNativeColumnarToRow (170) +- CometHashAggregate (169) +- CometExchange (168) +- CometHashAggregate (167) @@ -285,7 +285,7 @@ Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -(25) CometColumnarToRow [codegen id : 8] +(25) CometNativeColumnarToRow Input [1]: [h8_30_to_9#14] (26) CometNativeScan parquet spark_catalog.default.store_sales @@ -368,14 +368,14 @@ Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [h9_to_9_30#25] (45) BroadcastExchange Input [1]: [h9_to_9_30#25] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 8] +(46) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -459,14 +459,14 @@ Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -(65) CometColumnarToRow [codegen id : 2] +(65) CometNativeColumnarToRow Input [1]: [h9_30_to_10#36] (66) BroadcastExchange Input [1]: [h9_30_to_10#36] Arguments: IdentityBroadcastMode, [plan_id=5] -(67) BroadcastNestedLoopJoin [codegen id : 8] +(67) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -550,14 +550,14 @@ Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -(86) CometColumnarToRow [codegen id : 3] +(86) CometNativeColumnarToRow Input [1]: [h10_to_10_30#47] (87) BroadcastExchange Input [1]: [h10_to_10_30#47] Arguments: IdentityBroadcastMode, [plan_id=7] -(88) BroadcastNestedLoopJoin [codegen id : 8] +(88) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -641,14 +641,14 @@ Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -(107) CometColumnarToRow [codegen id : 4] +(107) CometNativeColumnarToRow Input [1]: [h10_30_to_11#58] (108) BroadcastExchange Input [1]: [h10_30_to_11#58] Arguments: IdentityBroadcastMode, [plan_id=9] -(109) BroadcastNestedLoopJoin [codegen id : 8] +(109) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -732,14 +732,14 @@ Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -(128) CometColumnarToRow [codegen id : 5] +(128) CometNativeColumnarToRow Input [1]: [h11_to_11_30#69] (129) BroadcastExchange Input [1]: [h11_to_11_30#69] Arguments: IdentityBroadcastMode, [plan_id=11] -(130) BroadcastNestedLoopJoin [codegen id : 8] +(130) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -823,14 +823,14 @@ Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -(149) CometColumnarToRow [codegen id : 6] +(149) CometNativeColumnarToRow Input [1]: [h11_30_to_12#80] (150) BroadcastExchange Input [1]: [h11_30_to_12#80] Arguments: IdentityBroadcastMode, [plan_id=13] -(151) BroadcastNestedLoopJoin [codegen id : 8] +(151) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None @@ -914,14 +914,14 @@ Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -(170) CometColumnarToRow [codegen id : 7] +(170) CometNativeColumnarToRow Input [1]: [h12_to_12_30#91] (171) BroadcastExchange Input [1]: [h12_to_12_30#91] Arguments: IdentityBroadcastMode, [plan_id=15] -(172) BroadcastNestedLoopJoin [codegen id : 8] +(172) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/extended.txt index 24f596de9b..87f19deeb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/extended.txt @@ -5,7 +5,7 @@ BroadcastNestedLoopJoin : : : :- BroadcastNestedLoopJoin : : : : :- BroadcastNestedLoopJoin : : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : : : :- CometColumnarToRow +: : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometHashAggregate : : : : : : : +- CometExchange : : : : : : : +- CometHashAggregate @@ -31,7 +31,7 @@ BroadcastNestedLoopJoin : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : : : +- BroadcastExchange -: : : : : : +- CometColumnarToRow +: : : : : : +- CometNativeColumnarToRow : : : : : : +- CometHashAggregate : : : : : : +- CometExchange : : : : : : +- CometHashAggregate @@ -57,7 +57,7 @@ BroadcastNestedLoopJoin : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : : +- BroadcastExchange -: : : : : +- CometColumnarToRow +: : : : : +- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometExchange : : : : : +- CometHashAggregate @@ -83,7 +83,7 @@ BroadcastNestedLoopJoin : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store : : : : +- BroadcastExchange -: : : : +- CometColumnarToRow +: : : : +- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometExchange : : : : +- CometHashAggregate @@ -109,7 +109,7 @@ BroadcastNestedLoopJoin : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- BroadcastExchange -: : : +- CometColumnarToRow +: : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -135,7 +135,7 @@ BroadcastNestedLoopJoin : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -161,7 +161,7 @@ BroadcastNestedLoopJoin : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange -: +- CometColumnarToRow +: +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -187,7 +187,7 @@ BroadcastNestedLoopJoin : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/simplified.txt index cef8e0e760..82809b3066 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/simplified.txt @@ -1,4 +1,4 @@ -WholeStageCodegen (8) +WholeStageCodegen (1) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin @@ -6,8 +6,8 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [count] [h8_30_to_9,count(1)] CometExchange #1 CometHashAggregate [count] @@ -34,57 +34,9 @@ WholeStageCodegen (8) CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 + CometNativeColumnarToRow + CometHashAggregate [count] [h9_to_9_30,count(1)] + CometExchange #6 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -96,18 +48,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #13 + CometBroadcastExchange [t_time_sk] #7 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 + InputAdapter + BroadcastExchange #8 + CometNativeColumnarToRow + CometHashAggregate [count] [h9_30_to_10,count(1)] + CometExchange #9 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -119,18 +69,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #16 + CometBroadcastExchange [t_time_sk] #10 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 + InputAdapter + BroadcastExchange #11 + CometNativeColumnarToRow + CometHashAggregate [count] [h10_to_10_30,count(1)] + CometExchange #12 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -142,18 +90,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #19 + CometBroadcastExchange [t_time_sk] #13 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 + InputAdapter + BroadcastExchange #14 + CometNativeColumnarToRow + CometHashAggregate [count] [h10_30_to_11,count(1)] + CometExchange #15 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -165,18 +111,16 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #22 + CometBroadcastExchange [t_time_sk] #16 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 + InputAdapter + BroadcastExchange #17 + CometNativeColumnarToRow + CometHashAggregate [count] [h11_to_11_30,count(1)] + CometExchange #18 CometHashAggregate [count] CometProject CometBroadcastHashJoin [ss_store_sk,s_store_sk] @@ -188,8 +132,50 @@ WholeStageCodegen (8) CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #25 + CometBroadcastExchange [t_time_sk] #19 CometProject [t_time_sk] CometFilter [t_time_sk,t_hour,t_minute] CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #20 + CometNativeColumnarToRow + CometHashAggregate [count] [h11_30_to_12,count(1)] + CometExchange #21 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #22 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #23 + CometNativeColumnarToRow + CometHashAggregate [count] [h12_to_12_30,count(1)] + CometExchange #24 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #25 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + ReusedExchange [s_store_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/explain.txt index 56583c6d26..2a3e4162b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/explain.txt @@ -3,11 +3,11 @@ TakeOrderedAndProject (30) +- * Project (29) +- * Filter (28) +- Window (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Project (19) @@ -16,7 +16,7 @@ TakeOrderedAndProject (30) : +- * BroadcastHashJoin Inner BuildRight (12) : :- * Project (10) : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) + : : :- CometNativeColumnarToRow (4) : : : +- CometProject (3) : : : +- CometFilter (2) : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -26,7 +26,7 @@ TakeOrderedAndProject (30) : : +- Scan parquet spark_catalog.default.store_sales (5) : +- ReusedExchange (11) +- BroadcastExchange (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometFilter (15) +- CometNativeScan parquet spark_catalog.default.store (14) @@ -46,7 +46,7 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) AS i_class#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) AS i_category#7] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] (5) Scan parquet spark_catalog.default.store_sales @@ -68,26 +68,26 @@ Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] (11) ReusedExchange [Reuses operator id: 35] Output [2]: [d_date_sk#13, d_moy#14] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#11] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14] Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#14] @@ -102,24 +102,24 @@ ReadSchema: struct Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] Condition : isnotnull(s_store_sk#15) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] (17) BroadcastExchange Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#9] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14, s_store_sk#15, s_store_name#16, s_company_name#17] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] @@ -130,10 +130,10 @@ Results [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_nam Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] @@ -148,18 +148,18 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#16, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#16 ASC NULLS FIRST, s_company_name#17 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] (27) Window Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#16, s_company_name#17, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#16, s_company_name#17] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END -(29) Project [codegen id : 7] +(29) Project [codegen id : 4] Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] @@ -171,7 +171,7 @@ Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometNativeScan parquet spark_catalog.default.date_dim (31) @@ -192,7 +192,7 @@ Condition : ((isnotnull(d_year#24) AND (d_year#24 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#13, d_year#24, d_moy#14] Arguments: [d_date_sk#13, d_moy#14], [d_date_sk#13, d_moy#14] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_moy#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt index 0b6c2edaa0..5c7a26057d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject +- Project +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -25,17 +25,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/simplified.txt index f87ef33db4..573b3eeaf0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/simplified.txt @@ -1,53 +1,47 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) + WholeStageCodegen (4) Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk,s_store_name,s_company_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt index 47ce0d4a91..801d3151d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_moy#15] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt index 61bfd1d960..e64a4daac6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt @@ -26,12 +26,10 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index 47ce0d4a91..801d3151d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -167,7 +167,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (34) -+- * CometColumnarToRow (33) ++- CometNativeColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_ Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] -(33) CometColumnarToRow [codegen id : 1] +(33) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_moy#15] (34) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt index 825b1ed81c..6b4c3e3383 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 61bfd1d960..e64a4daac6 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 @@ -26,12 +26,10 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_moy] + 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] CometBroadcastExchange [d_date_sk,d_moy] #5 CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/explain.txt index da194f2825..927b31b896 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * CometColumnarToRow (3) ++- CometNativeColumnarToRow (3) +- CometFilter (2) +- CometNativeScan parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) CometColumnarToRow [codegen id : 1] +(3) CometNativeColumnarToRow Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] @@ -26,7 +26,7 @@ Input [1]: [r_reason_sk#1] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) +CometNativeColumnarToRow (12) +- CometProject (11) +- CometHashAggregate (10) +- CometExchange (9) @@ -69,7 +69,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(Unscal Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] -(12) CometColumnarToRow [codegen id : 1] +(12) CometNativeColumnarToRow Input [1]: [mergedValue#29] Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] @@ -77,7 +77,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) +CometNativeColumnarToRow (20) +- CometProject (19) +- CometHashAggregate (18) +- CometExchange (17) @@ -120,7 +120,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(Unscal Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] -(20) CometColumnarToRow [codegen id : 1] +(20) CometNativeColumnarToRow Input [1]: [mergedValue#42] Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] @@ -128,7 +128,7 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) +CometNativeColumnarToRow (28) +- CometProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -171,7 +171,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(Unscal Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [mergedValue#55] Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] @@ -179,7 +179,7 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) +CometNativeColumnarToRow (36) +- CometProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -222,7 +222,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(Unscal Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [1]: [mergedValue#68] Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] @@ -230,7 +230,7 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) +CometNativeColumnarToRow (44) +- CometProject (43) +- CometHashAggregate (42) +- CometExchange (41) @@ -273,7 +273,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(Unscal Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [mergedValue#81] Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/extended.txt index 53ba3252cf..ff39852ed0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/extended.txt @@ -1,6 +1,6 @@ Project [COMET: ] : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -11,7 +11,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -22,7 +22,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -33,7 +33,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -44,7 +44,7 @@ : :- ReusedSubquery : :- ReusedSubquery : :- Subquery -: : +- CometColumnarToRow +: : +- CometNativeColumnarToRow : : +- CometProject : : +- CometHashAggregate : : +- CometExchange @@ -54,7 +54,7 @@ : : +- CometNativeScan parquet spark_catalog.default.store_sales : :- ReusedSubquery : +- ReusedSubquery -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/simplified.txt index 9593d6ebd1..813cb91b2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/simplified.txt @@ -1,71 +1,61 @@ WholeStageCodegen (1) Project Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #1 ReusedSubquery [mergedValue] #1 Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #2 ReusedSubquery [mergedValue] #2 Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #3 ReusedSubquery [mergedValue] #3 Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #4 ReusedSubquery [mergedValue] #4 Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - 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 [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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + CometNativeColumnarToRow + CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] + 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 [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] + CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [r_reason_sk] CometNativeScan parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/explain.txt index b3f32555de..14dc1ef68d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * CometColumnarToRow (3) ++- CometNativeColumnarToRow (3) +- CometFilter (2) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) CometColumnarToRow [codegen id : 1] +(3) CometNativeColumnarToRow Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt index ed71033b26..a86d85c820 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt @@ -54,7 +54,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : :- ReusedSubquery : +- ReusedSubquery -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/simplified.txt index 558f5f4b36..9b54130ef1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/simplified.txt @@ -65,7 +65,7 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [r_reason_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index b3f32555de..14dc1ef68d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == * Project (4) -+- * CometColumnarToRow (3) ++- CometNativeColumnarToRow (3) +- CometFilter (2) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) @@ -16,7 +16,7 @@ ReadSchema: struct Input [1]: [r_reason_sk#1] Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) -(3) CometColumnarToRow [codegen id : 1] +(3) CometNativeColumnarToRow Input [1]: [r_reason_sk#1] (4) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/extended.txt index ed71033b26..a86d85c820 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/extended.txt @@ -54,7 +54,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : :- ReusedSubquery : +- ReusedSubquery -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason 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 558f5f4b36..9b54130ef1 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 @@ -65,7 +65,7 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] ReusedSubquery [mergedValue] #5 ReusedSubquery [mergedValue] #5 - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [r_reason_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/explain.txt index 73b54f439b..71001884ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) + :- CometNativeColumnarToRow (25) : +- CometHashAggregate (24) : +- CometExchange (23) : +- CometHashAggregate (22) @@ -27,7 +27,7 @@ : +- CometFilter (17) : +- CometNativeScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * CometColumnarToRow (44) + +- CometNativeColumnarToRow (44) +- CometHashAggregate (43) +- CometExchange (42) +- CometHashAggregate (41) @@ -160,7 +160,7 @@ Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -(25) CometColumnarToRow [codegen id : 2] +(25) CometNativeColumnarToRow Input [1]: [amc#12] (26) CometNativeScan parquet spark_catalog.default.web_sales @@ -243,18 +243,18 @@ Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [pmc#22] (45) BroadcastExchange Input [1]: [pmc#22] Arguments: IdentityBroadcastMode, [plan_id=3] -(46) BroadcastNestedLoopJoin [codegen id : 2] +(46) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(47) Project [codegen id : 2] +(47) Project [codegen id : 1] Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] Input [2]: [amc#12, pmc#22] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/extended.txt index 977fcbbc6b..b1125e15e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/extended.txt @@ -1,6 +1,6 @@ Project +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -26,7 +26,7 @@ Project : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/simplified.txt index a13072cd91..686e05404c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/simplified.txt @@ -1,8 +1,8 @@ -WholeStageCodegen (2) +WholeStageCodegen (1) Project [amc,pmc] BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [count] [amc,count(1)] CometExchange #1 CometHashAggregate [count] @@ -29,24 +29,22 @@ WholeStageCodegen (2) CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] - CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] - CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - ReusedExchange [hd_demo_sk] #2 - CometBroadcastExchange [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - ReusedExchange [wp_web_page_sk] #4 + CometNativeColumnarToRow + CometHashAggregate [count] [pmc,count(1)] + CometExchange #6 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + CometProject [ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] + CometProject [ws_sold_time_sk,ws_web_page_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,hd_demo_sk] + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + ReusedExchange [hd_demo_sk] #2 + CometBroadcastExchange [t_time_sk] #7 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/explain.txt index b52840adb9..9ae2954307 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (47) +CometNativeColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -18,7 +18,7 @@ : : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : : :- * Project (10) : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * CometColumnarToRow (4) + : : : : : :- CometNativeColumnarToRow (4) : : : : : : +- CometProject (3) : : : : : : +- CometFilter (2) : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center (1) @@ -28,21 +28,21 @@ : : : : : +- Scan parquet spark_catalog.default.catalog_returns (5) : : : : +- ReusedExchange (11) : : : +- BroadcastExchange (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometFilter (15) : : : +- CometNativeScan parquet spark_catalog.default.customer (14) : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) + : : +- CometNativeColumnarToRow (23) : : +- CometProject (22) : : +- CometFilter (21) : : +- CometNativeScan parquet spark_catalog.default.customer_address (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_demographics (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.household_demographics (34) @@ -63,7 +63,7 @@ Condition : isnotnull(cc_call_center_sk#1) Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cc_call_center_id#2, 16, true, false, true) AS cc_call_center_id#5, cc_name#3, cc_manager#4] -(4) CometColumnarToRow [codegen id : 7] +(4) CometNativeColumnarToRow Input [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] (5) Scan parquet spark_catalog.default.catalog_returns @@ -85,26 +85,26 @@ Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_ Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cc_call_center_sk#1] Right keys [1]: [cr_call_center_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 2] Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] (11) ReusedExchange [Reuses operator id: 52] Output [1]: [d_date_sk#11] -(12) BroadcastHashJoin [codegen id : 7] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cr_returned_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 7] +(13) Project [codegen id : 2] Output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] @@ -119,20 +119,20 @@ ReadSchema: struct -(2) ColumnarToRow [codegen id : 6] +(2) ColumnarToRow [codegen id : 3] Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 6] +(3) Filter [codegen id : 3] Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) @@ -62,20 +62,20 @@ Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotn Input [2]: [i_item_sk#5, i_manufact_id#6] Arguments: [i_item_sk#5], [i_item_sk#5] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [i_item_sk#5] (8) BroadcastExchange Input [1]: [i_item_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 6] +(9) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 6] +(10) Project [codegen id : 3] Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] @@ -87,27 +87,27 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] +(12) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -(13) Filter [codegen id : 3] +(13) Filter [codegen id : 1] Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] Condition : isnotnull(ws_item_sk#7) (14) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#10] -(15) BroadcastHashJoin [codegen id : 3] +(15) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(16) Project [codegen id : 3] +(16) Project [codegen id : 1] Output [2]: [ws_item_sk#7, ws_ext_discount_amt#8] Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#10] -(17) HashAggregate [codegen id : 3] +(17) HashAggregate [codegen id : 1] Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] Keys [1]: [ws_item_sk#7] Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] @@ -118,17 +118,17 @@ Results [3]: [ws_item_sk#7, sum#13, count#14] Input [3]: [ws_item_sk#7, sum#13, count#14] Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(19) CometColumnarToRow [codegen id : 4] +(19) CometNativeColumnarToRow Input [3]: [ws_item_sk#7, sum#13, count#14] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#7, sum#13, count#14] Keys [1]: [ws_item_sk#7] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#15] Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -(21) Filter [codegen id : 4] +(21) Filter [codegen id : 2] Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) @@ -136,30 +136,30 @@ Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] -(23) BroadcastHashJoin [codegen id : 6] +(23) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_item_sk#5] Right keys [1]: [ws_item_sk#7] Join type: Inner Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#16) -(24) Project [codegen id : 6] +(24) Project [codegen id : 3] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] (25) ReusedExchange [Reuses operator id: 36] Output [1]: [d_date_sk#10] -(26) BroadcastHashJoin [codegen id : 6] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(27) Project [codegen id : 6] +(27) Project [codegen id : 3] Output [1]: [ws_ext_discount_amt#2] Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#10] -(28) HashAggregate [codegen id : 6] +(28) HashAggregate [codegen id : 3] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] @@ -170,10 +170,10 @@ Results [1]: [sum#18] Input [1]: [sum#18] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [1]: [sum#18] -(31) HashAggregate [codegen id : 7] +(31) HashAggregate [codegen id : 4] Input [1]: [sum#18] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] @@ -184,7 +184,7 @@ Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#19,17,2) AS Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (36) -+- * CometColumnarToRow (35) ++- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.date_dim (32) @@ -205,7 +205,7 @@ Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 2000-01-27)) AND (d_date#2 Input [2]: [d_date_sk#10, d_date#21] Arguments: [d_date_sk#10], [d_date_sk#10] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (36) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt index 2d15266049..b3ca85cdaf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt @@ -1,5 +1,5 @@ HashAggregate -+- CometColumnarToRow ++- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -13,19 +13,19 @@ HashAggregate : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -35,12 +35,12 @@ HashAggregate : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/simplified.txt index 17df728966..71aeded80b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/simplified.txt @@ -1,9 +1,9 @@ -WholeStageCodegen (7) +WholeStageCodegen (4) HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #1 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [ws_ext_discount_amt] [sum,sum] Project [ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] @@ -17,29 +17,25 @@ WholeStageCodegen (7) Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometFilter [i_item_sk,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Filter [(1.3 * avg(ws_ext_discount_amt))] HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ws_item_sk] #5 - WholeStageCodegen (3) + WholeStageCodegen (1) HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] Project [ws_item_sk,ws_ext_discount_amt] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/explain.txt index c317a12697..f99d6477e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/explain.txt @@ -175,7 +175,7 @@ Input [1]: [Excess Discount Amount #17] Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt index 20df9a8047..ca50f78e18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/simplified.txt index 4ec5755df7..991100027b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/simplified.txt @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index c317a12697..f99d6477e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -175,7 +175,7 @@ Input [1]: [Excess Discount Amount #17] Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (35) -+- * CometColumnarToRow (34) ++- CometNativeColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(34) CometColumnarToRow [codegen id : 1] +(34) CometNativeColumnarToRow Input [1]: [d_date_sk#11] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/extended.txt index 20df9a8047..ca50f78e18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 4ec5755df7..991100027b 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 @@ -14,12 +14,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk] #3 CometProject [i_item_sk] CometFilter [i_item_sk,i_manufact_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/explain.txt index 3f38adcd39..6a067b40ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (22) +CometNativeColumnarToRow (22) +- CometTakeOrderedAndProject (21) +- CometHashAggregate (20) +- CometExchange (19) @@ -119,6 +119,6 @@ Functions [1]: [sum(act_sales#14)] Input [2]: [ss_customer_sk#2, sumsales#17] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] -(22) CometColumnarToRow [codegen id : 1] +(22) CometNativeColumnarToRow Input [2]: [ss_customer_sk#2, sumsales#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/extended.txt index 7203a678b0..1496f26233 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/simplified.txt index c0b74010df..f08009ad54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/simplified.txt @@ -1,24 +1,22 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - 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] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] +CometNativeColumnarToRow + CometTakeOrderedAndProject [ss_customer_sk,sumsales] + CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] + CometExchange [ss_customer_sk] #1 + 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] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometExchange [ss_item_sk,ss_ticket_number] #2 + CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometExchange [sr_item_sk,sr_ticket_number] #3 + CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + CometBroadcastExchange [r_reason_sk] #4 + CometProject [r_reason_sk] + CometFilter [r_reason_sk,r_reason_desc] + CometNativeScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/explain.txt index 1caeb8ca76..b09a5a035c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (41) +CometNativeColumnarToRow (41) +- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) - +- * CometColumnarToRow (36) + +- CometNativeColumnarToRow (36) +- CometHashAggregate (35) +- CometProject (34) +- CometBroadcastHashJoin (33) @@ -204,7 +204,7 @@ Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -(36) CometColumnarToRow [codegen id : 1] +(36) CometNativeColumnarToRow Input [3]: [ws_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -230,6 +230,6 @@ Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -(41) CometColumnarToRow [codegen id : 2] +(41) CometNativeColumnarToRow Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/extended.txt index e5c7b8688a..c91f0410ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/extended.txt @@ -1,9 +1,9 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/simplified.txt index feab73bcd6..cd992a9669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/simplified.txt @@ -1,45 +1,43 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_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] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_warehouse_sk] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] +CometNativeColumnarToRow + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + 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] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number,ws_warehouse_sk] + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometExchange [ws_order_number] #2 + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometSort [ws_warehouse_sk,ws_order_number] + CometExchange [ws_order_number] #3 + CometProject [ws_warehouse_sk,ws_order_number] + CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometSort [wr_order_number] + CometExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/explain.txt index e49aacd1b5..60fac7e470 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/explain.txt @@ -1,10 +1,10 @@ == Physical Plan == -* CometColumnarToRow (54) +CometNativeColumnarToRow (54) +- CometHashAggregate (53) +- CometColumnarExchange (52) +- * HashAggregate (51) +- * HashAggregate (50) - +- * CometColumnarToRow (49) + +- CometNativeColumnarToRow (49) +- CometHashAggregate (48) +- CometProject (47) +- CometBroadcastHashJoin (46) @@ -271,7 +271,7 @@ Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] Keys [1]: [ws_order_number#4] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [3]: [ws_order_number#4, sum#21, sum#22] (50) HashAggregate [codegen id : 1] @@ -297,6 +297,6 @@ Input [3]: [sum#21, sum#22, count#26] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -(54) CometColumnarToRow [codegen id : 2] +(54) CometNativeColumnarToRow Input [3]: [order count #27, total shipping cost #28, total net profit #29] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/extended.txt index cbba03d28d..94e8b8d087 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/extended.txt @@ -1,9 +1,9 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometHashAggregate +- CometProject +- CometBroadcastHashJoin diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/simplified.txt index 15f29c507c..b81cc72852 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/simplified.txt @@ -1,58 +1,56 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_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] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] +CometNativeColumnarToRow + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + 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] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometExchange [ws_order_number] #2 + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometFilter [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + CometProject [ws_order_number] + CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] + CometSort [ws_warehouse_sk,ws_order_number] + CometExchange [ws_order_number] #3 + CometProject [ws_warehouse_sk,ws_order_number] + CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometProject [wr_order_number] + CometSortMergeJoin [wr_order_number,ws_order_number] + CometSort [wr_order_number] + CometExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometFilter [wr_order_number,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + CometProject [ws_order_number] + CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + CometProject [web_site_sk] + CometFilter [web_site_sk,web_company_name] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/explain.txt index db701aae34..7ca69ec813 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (25) +CometNativeColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -138,6 +138,6 @@ Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -(25) CometColumnarToRow [codegen id : 1] +(25) CometNativeColumnarToRow Input [1]: [count(1)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/extended.txt index 170d3b7879..9ee165830a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/simplified.txt index b40f41659f..6599ccccb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/simplified.txt @@ -1,27 +1,25 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] - CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] - CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] +CometNativeColumnarToRow + CometHashAggregate [count] [count(1),count(1)] + CometExchange #1 + CometHashAggregate [count] + CometProject + CometBroadcastHashJoin [ss_store_sk,s_store_sk] + CometProject [ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] + CometProject [ss_sold_time_sk,ss_store_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,hd_demo_sk] + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + CometBroadcastExchange [hd_demo_sk] #2 + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_dep_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + CometBroadcastExchange [t_time_sk] #3 + CometProject [t_time_sk] + CometFilter [t_time_sk,t_hour,t_minute] + CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + CometBroadcastExchange [s_store_sk] #4 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_store_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/explain.txt index a7f1cb451d..0121fd183f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (24) +CometNativeColumnarToRow (24) +- CometHashAggregate (23) +- CometExchange (22) +- CometHashAggregate (21) @@ -32,23 +32,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] (3) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] -(4) BroadcastHashJoin [codegen id : 2] +(4) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(5) Project [codegen id : 2] +(5) Project [codegen id : 1] Output [2]: [ss_item_sk#1, ss_customer_sk#2] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] -(6) HashAggregate [codegen id : 2] +(6) HashAggregate [codegen id : 1] Input [2]: [ss_item_sk#1, ss_customer_sk#2] Keys [2]: [ss_customer_sk#2, ss_item_sk#1] Functions: [] @@ -75,23 +75,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 2] Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] (12) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#11] -(13) BroadcastHashJoin [codegen id : 4] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#10] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(14) Project [codegen id : 4] +(14) Project [codegen id : 2] Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#11] -(15) HashAggregate [codegen id : 4] +(15) HashAggregate [codegen id : 2] Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] Functions: [] @@ -134,14 +134,14 @@ Input [3]: [sum#14, sum#15, sum#16] Keys: [] Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END)] -(24) CometColumnarToRow [codegen id : 5] +(24) CometNativeColumnarToRow Input [3]: [store_only#17, catalog_only#18, store_and_catalog#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -162,7 +162,7 @@ Condition : (((isnotnull(d_month_seq#20) AND (d_month_seq#20 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#20] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt index 798bfdaf1b..e3b777a05a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -14,12 +14,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -33,7 +33,7 @@ CometColumnarToRow : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/simplified.txt index dac440b122..04cb3e457d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/simplified.txt @@ -1,41 +1,37 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - 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 [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] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (4) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 +CometNativeColumnarToRow + 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 [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] + CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] + CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_customer_sk,ss_item_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + CometSort [customer_sk,item_sk] + CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] + CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 + WholeStageCodegen (2) + HashAggregate [cs_bill_customer_sk,cs_item_sk] + Project [cs_bill_customer_sk,cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/explain.txt index 420f37fccc..a21fa2b802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/explain.txt @@ -146,7 +146,7 @@ Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt index e1aefb8382..0969a0e796 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/simplified.txt index a7de047b25..d37700ec0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/simplified.txt @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index 420f37fccc..a21fa2b802 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -146,7 +146,7 @@ Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/extended.txt index e1aefb8382..0969a0e796 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/extended.txt @@ -13,7 +13,7 @@ CometColumnarToRow : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 a7de047b25..d37700ec0f 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 @@ -15,12 +15,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/explain.txt index 8263680b2b..dcc1ecf5c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/explain.txt @@ -1,15 +1,15 @@ == Physical Plan == -* CometColumnarToRow (26) +CometNativeColumnarToRow (26) +- CometProject (25) +- CometSort (24) +- CometColumnarExchange (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -20,7 +20,7 @@ : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.store_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -35,10 +35,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) @@ -57,37 +57,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 31] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -98,10 +98,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -116,14 +116,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] (21) Window Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] @@ -139,14 +139,14 @@ Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrev Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(26) CometColumnarToRow [codegen id : 7] +(26) CometNativeColumnarToRow Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometNativeScan parquet spark_catalog.default.date_dim (27) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt index 030031856f..af8cf29205 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt @@ -1,14 +1,14 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometProject +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -20,17 +20,17 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/simplified.txt index 3f64b57ffa..91f3f3c423 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/simplified.txt @@ -1,48 +1,40 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] +CometNativeColumnarToRow + CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + CometNativeColumnarToRow + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt index 4cc725ef2b..8ef0f76fe1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt @@ -147,7 +147,7 @@ Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrev Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt index 4c972848e7..152665febf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt index af05a7d8c6..8c213757be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt @@ -24,12 +24,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 4cc725ef2b..8ef0f76fe1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -147,7 +147,7 @@ Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrev Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -168,7 +168,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt index 4c972848e7..152665febf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 af05a7d8c6..8c213757be 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 @@ -24,12 +24,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/explain.txt index 3447b6efa3..0035984c4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (29) +CometNativeColumnarToRow (29) +- CometTakeOrderedAndProject (28) +- CometHashAggregate (27) +- CometExchange (26) @@ -163,6 +163,6 @@ Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) TH Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt index c075163849..c236a1647f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/simplified.txt index 98a215dbc6..4c6dcfd6fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/simplified.txt @@ -1,31 +1,29 @@ -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 [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 [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] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] - CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] - CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] - CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] +CometNativeColumnarToRow + 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 [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 [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] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_call_center_sk,cc_name] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name,sm_ship_mode_sk,sm_type] + CometProject [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + CometBroadcastHashJoin [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk,w_warehouse_sk,w_warehouse_name] + CometFilter [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #2 + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 + CometProject [sm_type] [sm_ship_mode_sk,sm_type] + CometFilter [sm_ship_mode_sk,sm_type] + CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + CometBroadcastExchange [cc_call_center_sk,cc_name] #4 + CometFilter [cc_call_center_sk,cc_name] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/explain.txt index 83b183d532..5ebf1115ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -11,7 +11,7 @@ TakeOrderedAndProject (43) : :- * Project (24) : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) + : : : :- CometNativeColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : +- BroadcastExchange (9) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- Scan parquet spark_catalog.default.catalog_sales (16) : : +- ReusedExchange (18) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer_address (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) @@ -55,7 +55,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] (6) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] @@ -85,7 +85,7 @@ Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi @@ -98,19 +98,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] (13) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#10] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#8 AS customer_sk#11] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] @@ -121,19 +121,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] +(17) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] (18) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#14] -(19) BroadcastHashJoin [codegen id : 6] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(20) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] @@ -143,13 +143,13 @@ Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] Input [1]: [customer_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#11] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 9] +(24) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -168,20 +168,20 @@ Condition : (ca_county#17 IN (Walker County,Richland County,Gaines County,Dougla Input [2]: [ca_address_sk#16, ca_county#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [1]: [ca_address_sk#16] (29) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(31) Project [codegen id : 9] +(31) Project [codegen id : 4] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] @@ -200,24 +200,24 @@ Condition : isnotnull(cd_demo_sk#18) Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#29, cd_purchase_estimate#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#23, 10, true, false, true) AS cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(35) CometColumnarToRow [codegen id : 8] +(35) CometNativeColumnarToRow Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] (36) BroadcastExchange Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 4] Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(39) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 4] Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [partial_count(1)] @@ -228,10 +228,10 @@ Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_pur Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 5] Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [count(1)] @@ -246,7 +246,7 @@ Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FI Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -267,7 +267,7 @@ Condition : (((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = Input [3]: [d_date_sk#7, d_year#40, d_moy#41] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt index 8aebe6be9a..9091536e96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : :- Project : : +- BroadcastHashJoin : : :- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange @@ -20,12 +20,12 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,7 +37,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -47,17 +47,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/simplified.txt index 11bfb6f6c3..5b1aad4ef8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,13 +13,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -27,18 +27,16 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 Union - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -47,7 +45,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -58,17 +56,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt index b6c8291e3b..77708a4cc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt @@ -232,7 +232,7 @@ Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36 Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt index 38f41d10a5..b54bf7704a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt index b6c8291e3b..77708a4cc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt @@ -232,7 +232,7 @@ Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36 Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 38f41d10a5..b54bf7704a 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/explain.txt index 207cd830ed..bbbac9673e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/explain.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject (79) : :- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -25,14 +25,14 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -44,14 +44,14 @@ TakeOrderedAndProject (79) : +- BroadcastExchange (56) : +- * Filter (55) : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (47) : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) + : : :- CometNativeColumnarToRow (41) : : : +- CometProject (40) : : : +- CometFilter (39) : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -62,14 +62,14 @@ TakeOrderedAndProject (79) : +- ReusedExchange (48) +- BroadcastExchange (76) +- * HashAggregate (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometColumnarExchange (73) +- * HashAggregate (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) :- * Project (68) : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) + : :- CometNativeColumnarToRow (62) : : +- CometProject (61) : : +- CometFilter (60) : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -95,7 +95,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] (5) Scan parquet spark_catalog.default.store_sales @@ -117,30 +117,30 @@ Condition : isnotnull(ss_customer_sk#15) Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#15] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] (11) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#20, d_year#21] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] @@ -151,17 +151,17 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#25, year_total#26] Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) @@ -180,7 +180,7 @@ Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] (23) Scan parquet spark_catalog.default.store_sales @@ -191,10 +191,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Condition : isnotnull(ss_customer_sk#35) @@ -202,30 +202,30 @@ Condition : isnotnull(ss_customer_sk#35) Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#27] Right keys [1]: [ss_customer_sk#35] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] (29) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#40, d_year#41] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#38] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] @@ -236,10 +236,10 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] @@ -250,7 +250,7 @@ Results [5]: [c_customer_id#9 AS customer_id#44, c_first_name#10 AS customer_fir Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#44] Join type: Inner @@ -271,7 +271,7 @@ Condition : (isnotnull(c_customer_sk#49) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] Arguments: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62], [c_customer_sk#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#51, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#52, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#53, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#55, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#56, 50, true, false, true) AS c_email_address#62] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [8]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62] (42) Scan parquet spark_catalog.default.web_sales @@ -282,10 +282,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] Condition : isnotnull(ws_bill_customer_sk#63) @@ -293,30 +293,30 @@ Condition : isnotnull(ws_bill_customer_sk#63) Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#49] Right keys [1]: [ws_bill_customer_sk#63] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] Input [12]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] (48) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#67, d_year#68] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#66] Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#68] Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66, d_date_sk#67, d_year#68] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#68] Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] @@ -327,17 +327,17 @@ Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cus Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#71] Results [2]: [c_customer_id#57 AS customer_id#72, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#71,18,2) AS year_total#73] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#72, year_total#73] Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) @@ -345,13 +345,13 @@ Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) Input [2]: [customer_id#72, year_total#73] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#72] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(58) Project [codegen id : 12] Output [8]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#73] Input [9]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, customer_id#72, year_total#73] @@ -370,7 +370,7 @@ Condition : (isnotnull(c_customer_sk#74) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] Arguments: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62], [c_customer_sk#74, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#75, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#76, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#77, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#78, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#79, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#80, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#81, 50, true, false, true) AS c_email_address#62] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [8]: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62] (63) Scan parquet spark_catalog.default.web_sales @@ -381,10 +381,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] Condition : isnotnull(ws_bill_customer_sk#82) @@ -392,30 +392,30 @@ Condition : isnotnull(ws_bill_customer_sk#82) Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#74] Right keys [1]: [ws_bill_customer_sk#82] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] Input [12]: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] (69) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#86, d_year#87] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#85] Right keys [1]: [d_date_sk#86] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#87] Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85, d_date_sk#86, d_year#87] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#87] Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))] @@ -426,10 +426,10 @@ Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cus Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))] @@ -440,13 +440,13 @@ Results [2]: [c_customer_id#57 AS customer_id#90, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#90, year_total#91] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#90] Join type: Inner Join condition: (CASE WHEN (year_total#73 > 0.00) THEN (year_total#91 / year_total#73) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#48 / year_total#26) ELSE 0E-20 END) -(78) Project [codegen id : 16] +(78) Project [codegen id : 12] Output [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] Input [10]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#73, customer_id#90, year_total#91] @@ -458,7 +458,7 @@ Arguments: 100, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULL Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometFilter (81) +- CometNativeScan parquet spark_catalog.default.date_dim (80) @@ -474,7 +474,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (83) BroadcastExchange @@ -483,7 +483,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 BroadcastExchange (87) -+- * CometColumnarToRow (86) ++- CometNativeColumnarToRow (86) +- CometFilter (85) +- CometNativeScan parquet spark_catalog.default.date_dim (84) @@ -499,7 +499,7 @@ ReadSchema: struct Input [2]: [d_date_sk#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(86) CometColumnarToRow [codegen id : 1] +(86) CometNativeColumnarToRow Input [2]: [d_date_sk#40, d_year#41] (87) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt index 0d59f7b833..5d218998be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -23,23 +23,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -49,24 +49,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -76,19 +76,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -98,7 +98,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/simplified.txt index 00a3e659d4..a15087828c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] @@ -7,17 +7,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] @@ -30,69 +30,65 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -102,25 +98,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt index 49c1c8e0d7..d4e6edd682 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#38, d_year#39] Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#38, d_year#39] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt index 157d1d587e..81d240b8dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt index 49c1c8e0d7..d4e6edd682 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt @@ -415,7 +415,7 @@ Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#38, d_year#39] Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#38, d_year#39] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 157d1d587e..81d240b8dd 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 @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/explain.txt index 2ebf9db593..bfc7f4c2e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.web_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) @@ -54,37 +54,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt index 6c2a775097..15648408fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/simplified.txt index c129b42cdb..2b82e74b78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt index 410aecff1b..aaf59c8747 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt index 58ac81ba0d..11c87950e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt index 410aecff1b..aaf59c8747 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 58ac81ba0d..11c87950e6 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/explain.txt index 59342eea7f..2d95b2b4cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (87) +- * BroadcastHashJoin Inner BuildRight (86) :- * Filter (68) : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) + : +- CometNativeColumnarToRow (66) : +- CometColumnarExchange (65) : +- * HashAggregate (64) : +- * Project (63) @@ -17,12 +17,12 @@ TakeOrderedAndProject (87) : : : +- BroadcastExchange (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) + : : : :- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (48) : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) + : : : :- CometNativeColumnarToRow (36) : : : : +- CometHashAggregate (35) : : : : +- CometColumnarExchange (34) : : : : +- * HashAggregate (33) @@ -35,7 +35,7 @@ TakeOrderedAndProject (87) : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : +- BroadcastExchange (27) : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) + : : : : : :- CometNativeColumnarToRow (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- BroadcastExchange (25) @@ -47,7 +47,7 @@ TakeOrderedAndProject (87) : : : : : : : +- * ColumnarToRow (14) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometNativeColumnarToRow (18) : : : : : : +- CometFilter (17) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : +- ReusedExchange (22) @@ -64,7 +64,7 @@ TakeOrderedAndProject (87) : : : +- ReusedExchange (43) : : +- BroadcastExchange (58) : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) + : : :- CometNativeColumnarToRow (55) : : : +- CometFilter (54) : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : +- ReusedExchange (56) @@ -72,7 +72,7 @@ TakeOrderedAndProject (87) +- BroadcastExchange (85) +- * Filter (84) +- * HashAggregate (83) - +- * CometColumnarToRow (82) + +- CometNativeColumnarToRow (82) +- CometColumnarExchange (81) +- * HashAggregate (80) +- * Project (79) @@ -96,10 +96,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -114,7 +114,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -143,7 +143,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -172,33 +172,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -303,7 +303,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -313,13 +313,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -327,7 +327,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -344,13 +344,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -416,17 +416,17 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 50] +(70) ColumnarToRow [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(71) Filter [codegen id : 50] +(71) Filter [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) (72) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#59] -(73) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#59] Join type: LeftSemi @@ -435,30 +435,30 @@ Join condition: None (74) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(75) BroadcastHashJoin [codegen id : 50] +(75) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#60] Join type: Inner Join condition: None -(76) Project [codegen id : 50] +(76) Project [codegen id : 28] Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] (77) ReusedExchange [Reuses operator id: 126] Output [1]: [d_date_sk#64] -(78) BroadcastHashJoin [codegen id : 50] +(78) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(79) Project [codegen id : 50] +(79) Project [codegen id : 28] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -(80) HashAggregate [codegen id : 50] +(80) HashAggregate [codegen id : 28] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] @@ -469,17 +469,17 @@ Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(82) CometColumnarToRow [codegen id : 51] +(82) CometNativeColumnarToRow Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -(83) HashAggregate [codegen id : 51] +(83) HashAggregate [codegen id : 29] Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71, count(1)#72] Results [6]: [store AS channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71 AS sales#74, count(1)#72 AS number_sales#75] -(84) Filter [codegen id : 51] +(84) Filter [codegen id : 29] Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -487,7 +487,7 @@ Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(Reu Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 52] +(86) BroadcastHashJoin [codegen id : 30] Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Join type: Inner @@ -501,7 +501,7 @@ Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * HashAggregate (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometColumnarExchange (105) +- * HashAggregate (104) +- Union (103) @@ -529,19 +529,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#12)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 2] +(89) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] (90) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#79] -(91) BroadcastHashJoin [codegen id : 2] +(91) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#78] Right keys [1]: [d_date_sk#79] Join type: Inner Join condition: None -(92) Project [codegen id : 2] +(92) Project [codegen id : 1] Output [2]: [ss_quantity#76 AS quantity#80, ss_list_price#77 AS list_price#81] Input [4]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#79] @@ -552,19 +552,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#12)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 4] +(94) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] (95) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#85] -(96) BroadcastHashJoin [codegen id : 4] +(96) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#84] Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(97) Project [codegen id : 4] +(97) Project [codegen id : 2] Output [2]: [cs_quantity#82 AS quantity#86, cs_list_price#83 AS list_price#87] Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#85] @@ -575,25 +575,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#12)] ReadSchema: struct -(99) ColumnarToRow [codegen id : 6] +(99) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] (100) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#91] -(101) BroadcastHashJoin [codegen id : 6] +(101) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#90] Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(102) Project [codegen id : 6] +(102) Project [codegen id : 3] Output [2]: [ws_quantity#88 AS quantity#92, ws_list_price#89 AS list_price#93] Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#91] (103) Union -(104) HashAggregate [codegen id : 7] +(104) HashAggregate [codegen id : 4] Input [2]: [quantity#80, list_price#81] Keys: [] Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] @@ -604,10 +604,10 @@ Results [2]: [sum#96, count#97] Input [2]: [sum#96, count#97] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(106) CometColumnarToRow [codegen id : 8] +(106) CometNativeColumnarToRow Input [2]: [sum#96, count#97] -(107) HashAggregate [codegen id : 8] +(107) HashAggregate [codegen id : 5] Input [2]: [sum#96, count#97] Keys: [] Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] @@ -622,7 +622,7 @@ Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (112) -+- * CometColumnarToRow (111) ++- CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometNativeScan parquet spark_catalog.default.date_dim (108) @@ -643,7 +643,7 @@ Condition : ((isnotnull(d_week_seq#100) AND (d_week_seq#100 = Subquery scalar-su Input [2]: [d_date_sk#40, d_week_seq#100] Arguments: [d_date_sk#40], [d_date_sk#40] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (112) BroadcastExchange @@ -651,7 +651,7 @@ Input [1]: [d_date_sk#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] Subquery:6 Hosting operator id = 109 Hosting Expression = Subquery scalar-subquery#101, [id=#102] -* CometColumnarToRow (116) +CometNativeColumnarToRow (116) +- CometProject (115) +- CometFilter (114) +- CometNativeScan parquet spark_catalog.default.date_dim (113) @@ -672,12 +672,12 @@ Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] Arguments: [d_week_seq#103], [d_week_seq#103] -(116) CometColumnarToRow [codegen id : 1] +(116) CometNativeColumnarToRow Input [1]: [d_week_seq#103] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometNativeScan parquet spark_catalog.default.date_dim (117) @@ -698,7 +698,7 @@ Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1998)) AND (d_year#107 < Input [2]: [d_date_sk#24, d_year#107] Arguments: [d_date_sk#24], [d_date_sk#24] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (121) BroadcastExchange @@ -713,7 +713,7 @@ Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquer Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 BroadcastExchange (126) -+- * CometColumnarToRow (125) ++- CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometNativeScan parquet spark_catalog.default.date_dim (122) @@ -734,7 +734,7 @@ Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = Subquery scalar-su Input [2]: [d_date_sk#64, d_week_seq#108] Arguments: [d_date_sk#64], [d_date_sk#64] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_date_sk#64] (126) BroadcastExchange @@ -742,7 +742,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:12 Hosting operator id = 123 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* CometColumnarToRow (130) +CometNativeColumnarToRow (130) +- CometProject (129) +- CometFilter (128) +- CometNativeScan parquet spark_catalog.default.date_dim (127) @@ -763,7 +763,7 @@ Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] Arguments: [d_week_seq#111], [d_week_seq#111] -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [1]: [d_week_seq#111] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt index c4138c254a..7921f2fa61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject :- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -23,7 +23,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -33,12 +33,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -51,11 +51,11 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,12 +63,12 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -81,13 +81,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -100,16 +100,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,28 +123,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -157,13 +157,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -176,16 +176,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -199,20 +199,20 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -221,7 +221,7 @@ TakeOrderedAndProject +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -234,11 +234,11 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -246,12 +246,12 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -264,13 +264,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -283,16 +283,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -306,28 +306,28 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -340,13 +340,13 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -359,16 +359,16 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -382,20 +382,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/simplified.txt index b0eae963c3..45d526a696 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (30) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #12 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -21,7 +21,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #6 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -56,37 +56,33 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (5) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [brand_id,class_id,category_id] CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -98,23 +94,21 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 - WholeStageCodegen (4) + WholeStageCodegen (2) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #8 - WholeStageCodegen (3) + WholeStageCodegen (1) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -126,18 +120,16 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 - WholeStageCodegen (9) + WholeStageCodegen (4) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -153,10 +145,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 - WholeStageCodegen (23) + WholeStageCodegen (13) BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter @@ -165,14 +157,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (51) + WholeStageCodegen (29) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + WholeStageCodegen (28) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -185,19 +177,15 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt index ea550c6c74..b235159855 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#99] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subq Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#104] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt index f56d229b68..c1b791f357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt @@ -44,11 +44,11 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,11 +197,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -224,7 +224,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,7 +288,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt index fb9abae378..71f9d72e7d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt @@ -39,19 +39,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -131,19 +125,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt index ea550c6c74..b235159855 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#99] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subq Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#104] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt index f56d229b68..c1b791f357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt @@ -44,11 +44,11 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,11 +197,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -224,7 +224,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,7 +288,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fb9abae378..71f9d72e7d 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 @@ -39,19 +39,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -131,19 +125,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/explain.txt index f6335b03dd..5c3ee44547 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/explain.txt @@ -1,18 +1,18 @@ == Physical Plan == -* CometColumnarToRow (139) +CometNativeColumnarToRow (139) +- CometTakeOrderedAndProject (138) +- CometHashAggregate (137) +- CometColumnarExchange (136) +- * HashAggregate (135) +- Union (134) :- * HashAggregate (105) - : +- * CometColumnarToRow (104) + : +- CometNativeColumnarToRow (104) : +- CometColumnarExchange (103) : +- * HashAggregate (102) : +- Union (101) : :- * Filter (68) : : +- * HashAggregate (67) - : : +- * CometColumnarToRow (66) + : : +- CometNativeColumnarToRow (66) : : +- CometColumnarExchange (65) : : +- * HashAggregate (64) : : +- * Project (63) @@ -26,12 +26,12 @@ : : : : +- BroadcastExchange (51) : : : : +- * Project (50) : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * CometColumnarToRow (6) + : : : : :- CometNativeColumnarToRow (6) : : : : : +- CometFilter (5) : : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : : +- BroadcastExchange (48) : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : :- * CometColumnarToRow (36) + : : : : :- CometNativeColumnarToRow (36) : : : : : +- CometHashAggregate (35) : : : : : +- CometColumnarExchange (34) : : : : : +- * HashAggregate (33) @@ -44,7 +44,7 @@ : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : : +- BroadcastExchange (27) : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * CometColumnarToRow (12) + : : : : : : :- CometNativeColumnarToRow (12) : : : : : : : +- CometFilter (11) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : : +- BroadcastExchange (25) @@ -56,7 +56,7 @@ : : : : : : : : +- * ColumnarToRow (14) : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometNativeColumnarToRow (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : : +- ReusedExchange (22) @@ -73,14 +73,14 @@ : : : : +- ReusedExchange (43) : : : +- BroadcastExchange (58) : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * CometColumnarToRow (55) + : : : :- CometNativeColumnarToRow (55) : : : : +- CometFilter (54) : : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : : +- ReusedExchange (56) : : +- ReusedExchange (61) : :- * Filter (84) : : +- * HashAggregate (83) - : : +- * CometColumnarToRow (82) + : : +- CometNativeColumnarToRow (82) : : +- CometColumnarExchange (81) : : +- * HashAggregate (80) : : +- * Project (79) @@ -96,7 +96,7 @@ : : +- ReusedExchange (77) : +- * Filter (100) : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) + : +- CometNativeColumnarToRow (98) : +- CometColumnarExchange (97) : +- * HashAggregate (96) : +- * Project (95) @@ -111,32 +111,32 @@ : : +- ReusedExchange (90) : +- ReusedExchange (93) :- * HashAggregate (112) - : +- * CometColumnarToRow (111) + : +- CometNativeColumnarToRow (111) : +- CometColumnarExchange (110) : +- * HashAggregate (109) : +- * HashAggregate (108) - : +- * CometColumnarToRow (107) + : +- CometNativeColumnarToRow (107) : +- ReusedExchange (106) :- * HashAggregate (119) - : +- * CometColumnarToRow (118) + : +- CometNativeColumnarToRow (118) : +- CometColumnarExchange (117) : +- * HashAggregate (116) : +- * HashAggregate (115) - : +- * CometColumnarToRow (114) + : +- CometNativeColumnarToRow (114) : +- ReusedExchange (113) :- * HashAggregate (126) - : +- * CometColumnarToRow (125) + : +- CometNativeColumnarToRow (125) : +- CometColumnarExchange (124) : +- * HashAggregate (123) : +- * HashAggregate (122) - : +- * CometColumnarToRow (121) + : +- CometNativeColumnarToRow (121) : +- ReusedExchange (120) +- * HashAggregate (133) - +- * CometColumnarToRow (132) + +- CometNativeColumnarToRow (132) +- CometColumnarExchange (131) +- * HashAggregate (130) +- * HashAggregate (129) - +- * CometColumnarToRow (128) + +- CometNativeColumnarToRow (128) +- ReusedExchange (127) @@ -148,10 +148,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -166,7 +166,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -195,7 +195,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -224,33 +224,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 174] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -355,7 +355,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -365,13 +365,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -379,7 +379,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -396,13 +396,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -468,17 +468,17 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 51] +(70) ColumnarToRow [codegen id : 29] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -(71) Filter [codegen id : 51] +(71) Filter [codegen id : 29] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Condition : isnotnull(cs_item_sk#54) (72) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#58] -(73) BroadcastHashJoin [codegen id : 51] +(73) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#54] Right keys [1]: [ss_item_sk#58] Join type: LeftSemi @@ -487,30 +487,30 @@ Join condition: None (74) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(75) BroadcastHashJoin [codegen id : 51] +(75) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#54] Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(76) Project [codegen id : 51] +(76) Project [codegen id : 29] Output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] (77) ReusedExchange [Reuses operator id: 169] Output [1]: [d_date_sk#63] -(78) BroadcastHashJoin [codegen id : 51] +(78) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_sold_date_sk#57] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(79) Project [codegen id : 51] +(79) Project [codegen id : 29] Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] -(80) HashAggregate [codegen id : 51] +(80) HashAggregate [codegen id : 29] Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] @@ -521,17 +521,17 @@ Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(82) CometColumnarToRow [codegen id : 52] +(82) CometNativeColumnarToRow Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -(83) HashAggregate [codegen id : 52] +(83) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#70, count(1)#71] Results [6]: [catalog AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(84) Filter [codegen id : 52] +(84) Filter [codegen id : 30] Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -543,17 +543,17 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 77] +(86) ColumnarToRow [codegen id : 44] Input [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] -(87) Filter [codegen id : 77] +(87) Filter [codegen id : 44] Input [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] Condition : isnotnull(ws_item_sk#75) (88) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#79] -(89) BroadcastHashJoin [codegen id : 77] +(89) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#75] Right keys [1]: [ss_item_sk#79] Join type: LeftSemi @@ -562,30 +562,30 @@ Join condition: None (90) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(91) BroadcastHashJoin [codegen id : 77] +(91) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#75] Right keys [1]: [i_item_sk#80] Join type: Inner Join condition: None -(92) Project [codegen id : 77] +(92) Project [codegen id : 44] Output [6]: [ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_brand_id#81, i_class_id#82, i_category_id#83] Input [8]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] (93) ReusedExchange [Reuses operator id: 169] Output [1]: [d_date_sk#84] -(94) BroadcastHashJoin [codegen id : 77] +(94) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#78] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(95) Project [codegen id : 77] +(95) Project [codegen id : 44] Output [5]: [ws_quantity#76, ws_list_price#77, i_brand_id#81, i_class_id#82, i_category_id#83] Input [7]: [ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_brand_id#81, i_class_id#82, i_category_id#83, d_date_sk#84] -(96) HashAggregate [codegen id : 77] +(96) HashAggregate [codegen id : 44] Input [5]: [ws_quantity#76, ws_list_price#77, i_brand_id#81, i_class_id#82, i_category_id#83] Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] Functions [2]: [partial_sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77)), partial_count(1)] @@ -596,23 +596,23 @@ Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89 Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(98) CometColumnarToRow [codegen id : 78] +(98) CometNativeColumnarToRow Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] -(99) HashAggregate [codegen id : 78] +(99) HashAggregate [codegen id : 45] Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] Functions [2]: [sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77))#91, count(1)#92] Results [6]: [web AS channel#93, i_brand_id#81, i_class_id#82, i_category_id#83, sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77))#91 AS sales#94, count(1)#92 AS number_sales#95] -(100) Filter [codegen id : 78] +(100) Filter [codegen id : 45] Input [6]: [channel#93, i_brand_id#81, i_class_id#82, i_category_id#83, sales#94, number_sales#95] Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) (101) Union -(102) HashAggregate [codegen id : 79] +(102) HashAggregate [codegen id : 46] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] @@ -623,10 +623,10 @@ Results [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99 Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(104) CometColumnarToRow [codegen id : 80] +(104) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] -(105) HashAggregate [codegen id : 80] +(105) HashAggregate [codegen id : 47] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] @@ -636,17 +636,17 @@ Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(sa (106) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#49, i_brand_id#106, i_class_id#107, i_category_id#108, sum#99, isEmpty#100, sum#101] -(107) CometColumnarToRow [codegen id : 160] +(107) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#106, i_class_id#107, i_category_id#108, sum#99, isEmpty#100, sum#101] -(108) HashAggregate [codegen id : 160] +(108) HashAggregate [codegen id : 94] Input [7]: [channel#49, i_brand_id#106, i_class_id#107, i_category_id#108, sum#99, isEmpty#100, sum#101] Keys [4]: [channel#49, i_brand_id#106, i_class_id#107, i_category_id#108] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] Results [5]: [channel#49, i_brand_id#106, i_class_id#107, sum(sales#50)#102 AS sum_sales#109, sum(number_sales#51)#103 AS number_sales#110] -(109) HashAggregate [codegen id : 160] +(109) HashAggregate [codegen id : 94] Input [5]: [channel#49, i_brand_id#106, i_class_id#107, sum_sales#109, number_sales#110] Keys [3]: [channel#49, i_brand_id#106, i_class_id#107] Functions [2]: [partial_sum(sum_sales#109), partial_sum(number_sales#110)] @@ -657,10 +657,10 @@ Results [6]: [channel#49, i_brand_id#106, i_class_id#107, sum#114, isEmpty#115, Input [6]: [channel#49, i_brand_id#106, i_class_id#107, sum#114, isEmpty#115, sum#116] Arguments: hashpartitioning(channel#49, i_brand_id#106, i_class_id#107, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(111) CometColumnarToRow [codegen id : 161] +(111) CometNativeColumnarToRow Input [6]: [channel#49, i_brand_id#106, i_class_id#107, sum#114, isEmpty#115, sum#116] -(112) HashAggregate [codegen id : 161] +(112) HashAggregate [codegen id : 95] Input [6]: [channel#49, i_brand_id#106, i_class_id#107, sum#114, isEmpty#115, sum#116] Keys [3]: [channel#49, i_brand_id#106, i_class_id#107] Functions [2]: [sum(sum_sales#109), sum(number_sales#110)] @@ -670,17 +670,17 @@ Results [6]: [channel#49, i_brand_id#106, i_class_id#107, null AS i_category_id# (113) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#49, i_brand_id#122, i_class_id#123, i_category_id#124, sum#99, isEmpty#100, sum#101] -(114) CometColumnarToRow [codegen id : 241] +(114) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#122, i_class_id#123, i_category_id#124, sum#99, isEmpty#100, sum#101] -(115) HashAggregate [codegen id : 241] +(115) HashAggregate [codegen id : 142] Input [7]: [channel#49, i_brand_id#122, i_class_id#123, i_category_id#124, sum#99, isEmpty#100, sum#101] Keys [4]: [channel#49, i_brand_id#122, i_class_id#123, i_category_id#124] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] Results [4]: [channel#49, i_brand_id#122, sum(sales#50)#102 AS sum_sales#125, sum(number_sales#51)#103 AS number_sales#126] -(116) HashAggregate [codegen id : 241] +(116) HashAggregate [codegen id : 142] Input [4]: [channel#49, i_brand_id#122, sum_sales#125, number_sales#126] Keys [2]: [channel#49, i_brand_id#122] Functions [2]: [partial_sum(sum_sales#125), partial_sum(number_sales#126)] @@ -691,10 +691,10 @@ Results [5]: [channel#49, i_brand_id#122, sum#130, isEmpty#131, sum#132] Input [5]: [channel#49, i_brand_id#122, sum#130, isEmpty#131, sum#132] Arguments: hashpartitioning(channel#49, i_brand_id#122, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(118) CometColumnarToRow [codegen id : 242] +(118) CometNativeColumnarToRow Input [5]: [channel#49, i_brand_id#122, sum#130, isEmpty#131, sum#132] -(119) HashAggregate [codegen id : 242] +(119) HashAggregate [codegen id : 143] Input [5]: [channel#49, i_brand_id#122, sum#130, isEmpty#131, sum#132] Keys [2]: [channel#49, i_brand_id#122] Functions [2]: [sum(sum_sales#125), sum(number_sales#126)] @@ -704,17 +704,17 @@ Results [6]: [channel#49, i_brand_id#122, null AS i_class_id#135, null AS i_cate (120) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#49, i_brand_id#139, i_class_id#140, i_category_id#141, sum#99, isEmpty#100, sum#101] -(121) CometColumnarToRow [codegen id : 322] +(121) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#139, i_class_id#140, i_category_id#141, sum#99, isEmpty#100, sum#101] -(122) HashAggregate [codegen id : 322] +(122) HashAggregate [codegen id : 190] Input [7]: [channel#49, i_brand_id#139, i_class_id#140, i_category_id#141, sum#99, isEmpty#100, sum#101] Keys [4]: [channel#49, i_brand_id#139, i_class_id#140, i_category_id#141] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] Results [3]: [channel#49, sum(sales#50)#102 AS sum_sales#142, sum(number_sales#51)#103 AS number_sales#143] -(123) HashAggregate [codegen id : 322] +(123) HashAggregate [codegen id : 190] Input [3]: [channel#49, sum_sales#142, number_sales#143] Keys [1]: [channel#49] Functions [2]: [partial_sum(sum_sales#142), partial_sum(number_sales#143)] @@ -725,10 +725,10 @@ Results [4]: [channel#49, sum#147, isEmpty#148, sum#149] Input [4]: [channel#49, sum#147, isEmpty#148, sum#149] Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(125) CometColumnarToRow [codegen id : 323] +(125) CometNativeColumnarToRow Input [4]: [channel#49, sum#147, isEmpty#148, sum#149] -(126) HashAggregate [codegen id : 323] +(126) HashAggregate [codegen id : 191] Input [4]: [channel#49, sum#147, isEmpty#148, sum#149] Keys [1]: [channel#49] Functions [2]: [sum(sum_sales#142), sum(number_sales#143)] @@ -738,17 +738,17 @@ Results [6]: [channel#49, null AS i_brand_id#152, null AS i_class_id#153, null A (127) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#49, i_brand_id#157, i_class_id#158, i_category_id#159, sum#99, isEmpty#100, sum#101] -(128) CometColumnarToRow [codegen id : 403] +(128) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#157, i_class_id#158, i_category_id#159, sum#99, isEmpty#100, sum#101] -(129) HashAggregate [codegen id : 403] +(129) HashAggregate [codegen id : 238] Input [7]: [channel#49, i_brand_id#157, i_class_id#158, i_category_id#159, sum#99, isEmpty#100, sum#101] Keys [4]: [channel#49, i_brand_id#157, i_class_id#158, i_category_id#159] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] Results [2]: [sum(sales#50)#102 AS sum_sales#160, sum(number_sales#51)#103 AS number_sales#161] -(130) HashAggregate [codegen id : 403] +(130) HashAggregate [codegen id : 238] Input [2]: [sum_sales#160, number_sales#161] Keys: [] Functions [2]: [partial_sum(sum_sales#160), partial_sum(number_sales#161)] @@ -759,10 +759,10 @@ Results [3]: [sum#165, isEmpty#166, sum#167] Input [3]: [sum#165, isEmpty#166, sum#167] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(132) CometColumnarToRow [codegen id : 404] +(132) CometNativeColumnarToRow Input [3]: [sum#165, isEmpty#166, sum#167] -(133) HashAggregate [codegen id : 404] +(133) HashAggregate [codegen id : 239] Input [3]: [sum#165, isEmpty#166, sum#167] Keys: [] Functions [2]: [sum(sum_sales#160), sum(number_sales#161)] @@ -771,7 +771,7 @@ Results [6]: [null AS channel#170, null AS i_brand_id#171, null AS i_class_id#17 (134) Union -(135) HashAggregate [codegen id : 405] +(135) HashAggregate [codegen id : 240] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] Functions: [] @@ -791,14 +791,14 @@ Functions: [] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#37 ASC NULLS FIRST,i_class_id#38 ASC NULLS FIRST,i_category_id#39 ASC NULLS FIRST], output=[channel#49,i_brand_id#37,i_class_id#38,i_category_id#39,sum_sales#104,number_sales#105]), [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -(139) CometColumnarToRow [codegen id : 406] +(139) CometNativeColumnarToRow Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] ===== Subqueries ===== Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * HashAggregate (159) -+- * CometColumnarToRow (158) ++- CometNativeColumnarToRow (158) +- CometColumnarExchange (157) +- * HashAggregate (156) +- Union (155) @@ -826,19 +826,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#178), dynamicpruningexpression(ss_sold_date_sk#178 IN dynamicpruning#12)] ReadSchema: struct -(141) ColumnarToRow [codegen id : 2] +(141) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#176, ss_list_price#177, ss_sold_date_sk#178] (142) ReusedExchange [Reuses operator id: 174] Output [1]: [d_date_sk#179] -(143) BroadcastHashJoin [codegen id : 2] +(143) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#178] Right keys [1]: [d_date_sk#179] Join type: Inner Join condition: None -(144) Project [codegen id : 2] +(144) Project [codegen id : 1] Output [2]: [ss_quantity#176 AS quantity#180, ss_list_price#177 AS list_price#181] Input [4]: [ss_quantity#176, ss_list_price#177, ss_sold_date_sk#178, d_date_sk#179] @@ -849,19 +849,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#184), dynamicpruningexpression(cs_sold_date_sk#184 IN dynamicpruning#185)] ReadSchema: struct -(146) ColumnarToRow [codegen id : 4] +(146) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#182, cs_list_price#183, cs_sold_date_sk#184] (147) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#186] -(148) BroadcastHashJoin [codegen id : 4] +(148) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#184] Right keys [1]: [d_date_sk#186] Join type: Inner Join condition: None -(149) Project [codegen id : 4] +(149) Project [codegen id : 2] Output [2]: [cs_quantity#182 AS quantity#187, cs_list_price#183 AS list_price#188] Input [4]: [cs_quantity#182, cs_list_price#183, cs_sold_date_sk#184, d_date_sk#186] @@ -872,25 +872,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#191), dynamicpruningexpression(ws_sold_date_sk#191 IN dynamicpruning#185)] ReadSchema: struct -(151) ColumnarToRow [codegen id : 6] +(151) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#189, ws_list_price#190, ws_sold_date_sk#191] (152) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#192] -(153) BroadcastHashJoin [codegen id : 6] +(153) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#191] Right keys [1]: [d_date_sk#192] Join type: Inner Join condition: None -(154) Project [codegen id : 6] +(154) Project [codegen id : 3] Output [2]: [ws_quantity#189 AS quantity#193, ws_list_price#190 AS list_price#194] Input [4]: [ws_quantity#189, ws_list_price#190, ws_sold_date_sk#191, d_date_sk#192] (155) Union -(156) HashAggregate [codegen id : 7] +(156) HashAggregate [codegen id : 4] Input [2]: [quantity#180, list_price#181] Keys: [] Functions [1]: [partial_avg((cast(quantity#180 as decimal(10,0)) * list_price#181))] @@ -901,10 +901,10 @@ Results [2]: [sum#197, count#198] Input [2]: [sum#197, count#198] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(158) CometColumnarToRow [codegen id : 8] +(158) CometNativeColumnarToRow Input [2]: [sum#197, count#198] -(159) HashAggregate [codegen id : 8] +(159) HashAggregate [codegen id : 5] Input [2]: [sum#197, count#198] Keys: [] Functions [1]: [avg((cast(quantity#180 as decimal(10,0)) * list_price#181))] @@ -915,7 +915,7 @@ Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#178 IN Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#184 IN dynamicpruning#185 BroadcastExchange (164) -+- * CometColumnarToRow (163) ++- CometNativeColumnarToRow (163) +- CometProject (162) +- CometFilter (161) +- CometNativeScan parquet spark_catalog.default.date_dim (160) @@ -936,7 +936,7 @@ Condition : (((isnotnull(d_year#201) AND (d_year#201 >= 1998)) AND (d_year#201 < Input [2]: [d_date_sk#186, d_year#201] Arguments: [d_date_sk#186], [d_date_sk#186] -(163) CometColumnarToRow [codegen id : 1] +(163) CometNativeColumnarToRow Input [1]: [d_date_sk#186] (164) BroadcastExchange @@ -947,7 +947,7 @@ Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#191 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (169) -+- * CometColumnarToRow (168) ++- CometNativeColumnarToRow (168) +- CometProject (167) +- CometFilter (166) +- CometNativeScan parquet spark_catalog.default.date_dim (165) @@ -968,7 +968,7 @@ Condition : ((((isnotnull(d_year#202) AND isnotnull(d_moy#203)) AND (d_year#202 Input [3]: [d_date_sk#40, d_year#202, d_moy#203] Arguments: [d_date_sk#40], [d_date_sk#40] -(168) CometColumnarToRow [codegen id : 1] +(168) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (169) BroadcastExchange @@ -977,7 +977,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (174) -+- * CometColumnarToRow (173) ++- CometNativeColumnarToRow (173) +- CometProject (172) +- CometFilter (171) +- CometNativeScan parquet spark_catalog.default.date_dim (170) @@ -998,7 +998,7 @@ Condition : (((isnotnull(d_year#204) AND (d_year#204 >= 1999)) AND (d_year#204 < Input [2]: [d_date_sk#24, d_year#204] Arguments: [d_date_sk#24], [d_date_sk#24] -(173) CometColumnarToRow [codegen id : 1] +(173) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (174) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt index c6478057d9..0bc07e401e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt @@ -1,18 +1,18 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,12 +32,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -47,12 +47,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -65,19 +65,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -90,13 +90,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -109,16 +109,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -132,28 +132,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -166,13 +166,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -185,16 +185,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -208,23 +208,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -239,12 +239,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -257,13 +257,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -276,16 +276,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -299,28 +299,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -333,13 +333,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -352,16 +352,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -375,23 +375,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -406,12 +406,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -424,13 +424,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -443,16 +443,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -466,28 +466,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -500,13 +500,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -519,16 +519,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -542,32 +542,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -577,7 +577,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -587,12 +587,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -602,12 +602,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -620,19 +620,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -645,13 +645,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -664,16 +664,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -687,28 +687,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -721,13 +721,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -740,16 +740,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -763,23 +763,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -794,12 +794,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -812,13 +812,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -831,16 +831,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -854,28 +854,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -888,13 +888,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -907,16 +907,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -930,23 +930,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -961,12 +961,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -979,13 +979,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -998,16 +998,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1021,28 +1021,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -1055,13 +1055,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -1074,16 +1074,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1097,32 +1097,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -1132,7 +1132,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1142,12 +1142,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1157,12 +1157,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1175,19 +1175,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1200,13 +1200,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1219,16 +1219,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1242,28 +1242,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1276,13 +1276,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1295,16 +1295,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1318,23 +1318,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1349,12 +1349,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1367,13 +1367,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1386,16 +1386,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1409,28 +1409,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1443,13 +1443,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1462,16 +1462,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1485,23 +1485,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -1516,12 +1516,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1534,13 +1534,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1553,16 +1553,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1576,28 +1576,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -1610,13 +1610,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -1629,16 +1629,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1652,32 +1652,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -1687,7 +1687,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1697,12 +1697,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1712,12 +1712,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1730,19 +1730,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1755,13 +1755,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1774,16 +1774,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1797,28 +1797,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1831,13 +1831,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1850,16 +1850,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1873,23 +1873,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1904,12 +1904,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1922,13 +1922,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1941,16 +1941,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1964,28 +1964,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1998,13 +1998,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2017,16 +2017,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2040,23 +2040,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -2071,12 +2071,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -2089,13 +2089,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2108,16 +2108,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2131,28 +2131,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2165,13 +2165,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2184,16 +2184,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2207,32 +2207,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -2242,7 +2242,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2252,12 +2252,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2267,12 +2267,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -2285,19 +2285,19 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -2310,13 +2310,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2329,16 +2329,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2352,28 +2352,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2386,13 +2386,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2405,16 +2405,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2428,23 +2428,23 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -2459,12 +2459,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -2477,13 +2477,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2496,16 +2496,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2519,28 +2519,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2553,13 +2553,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2572,16 +2572,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2595,23 +2595,23 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -2626,12 +2626,12 @@ CometColumnarToRow : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2644,13 +2644,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2663,16 +2663,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2686,28 +2686,28 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -2720,13 +2720,13 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -2739,16 +2739,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2762,16 +2762,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/simplified.txt index 4458723393..69a4e70593 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/simplified.txt @@ -1,276 +1,266 @@ -WholeStageCodegen (406) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + WholeStageCodegen (240) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Union + WholeStageCodegen (47) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (46) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (15) + Filter [sales] + Subquery #3 + WholeStageCodegen (5) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #14 + WholeStageCodegen (4) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (1) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (2) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #15 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (3) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #15 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (14) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + BroadcastExchange #6 + WholeStageCodegen (5) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + CometNativeColumnarToRow + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (3) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (2) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + BroadcastExchange #11 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (4) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #13 + WholeStageCodegen (13) + BroadcastHashJoin [i_item_sk,ss_item_sk] + InputAdapter + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (30) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (29) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (45) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (44) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id] #19 - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #20 - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #21 - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (95) + HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,i_brand_id,i_class_id] #18 + WholeStageCodegen (94) + HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (143) + HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,i_brand_id] #19 + WholeStageCodegen (142) + HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (191) + HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel] #20 + WholeStageCodegen (190) + HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (239) + HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #21 + WholeStageCodegen (238) + HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + 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/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt index a727c5e743..ead027e63f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt @@ -809,7 +809,7 @@ Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 BroadcastExchange (147) -+- * CometColumnarToRow (146) ++- CometNativeColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 < Input [2]: [d_date_sk#154, d_year#155] Arguments: [d_date_sk#154], [d_date_sk#154] -(146) CometColumnarToRow [codegen id : 1] +(146) CometNativeColumnarToRow Input [1]: [d_date_sk#154] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * CometColumnarToRow (151) ++- CometNativeColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) CometColumnarToRow [codegen id : 1] +(151) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * CometColumnarToRow (156) ++- CometNativeColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 < Input [2]: [d_date_sk#26, d_year#168] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) CometColumnarToRow [codegen id : 1] +(156) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (157) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt index 83deb45432..3bacf3c8bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -219,7 +219,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -283,7 +283,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -360,7 +360,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -424,7 +424,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -497,7 +497,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -525,7 +525,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -547,7 +547,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -611,7 +611,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -688,7 +688,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -752,7 +752,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -829,7 +829,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -893,7 +893,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -966,7 +966,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -994,7 +994,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1016,7 +1016,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1080,7 +1080,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1157,7 +1157,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1221,7 +1221,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1298,7 +1298,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1362,7 +1362,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1435,7 +1435,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1463,7 +1463,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1485,7 +1485,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1549,7 +1549,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1626,7 +1626,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1690,7 +1690,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1767,7 +1767,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1831,7 +1831,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1904,7 +1904,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1932,7 +1932,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1954,7 +1954,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2018,7 +2018,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2095,7 +2095,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2159,7 +2159,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2236,7 +2236,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2300,7 +2300,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt index 758e917335..26ab7f52d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #18 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -56,12 +54,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #5 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -80,12 +76,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt index a727c5e743..ead027e63f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt @@ -809,7 +809,7 @@ Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 BroadcastExchange (147) -+- * CometColumnarToRow (146) ++- CometNativeColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 < Input [2]: [d_date_sk#154, d_year#155] Arguments: [d_date_sk#154], [d_date_sk#154] -(146) CometColumnarToRow [codegen id : 1] +(146) CometNativeColumnarToRow Input [1]: [d_date_sk#154] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * CometColumnarToRow (151) ++- CometNativeColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) CometColumnarToRow [codegen id : 1] +(151) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * CometColumnarToRow (156) ++- CometNativeColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 < Input [2]: [d_date_sk#26, d_year#168] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) CometColumnarToRow [codegen id : 1] +(156) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (157) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt index 83deb45432..3bacf3c8bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt @@ -28,7 +28,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -219,7 +219,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -283,7 +283,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -360,7 +360,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -424,7 +424,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -497,7 +497,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -525,7 +525,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -547,7 +547,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -611,7 +611,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -688,7 +688,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -752,7 +752,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -829,7 +829,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -893,7 +893,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -966,7 +966,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -994,7 +994,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1016,7 +1016,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1080,7 +1080,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1157,7 +1157,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1221,7 +1221,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1298,7 +1298,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1362,7 +1362,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1435,7 +1435,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1463,7 +1463,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1485,7 +1485,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1549,7 +1549,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1626,7 +1626,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1690,7 +1690,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1767,7 +1767,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1831,7 +1831,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1904,7 +1904,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1932,7 +1932,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1954,7 +1954,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2018,7 +2018,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2095,7 +2095,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2159,7 +2159,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2236,7 +2236,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2300,7 +2300,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 758e917335..26ab7f52d4 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 @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #18 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -56,12 +54,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #5 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -80,12 +76,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/explain.txt index db9dbcc51a..496a6370d9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (161) +- Union (160) :- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -21,32 +21,32 @@ TakeOrderedAndProject (161) : : : : : : : +- * ColumnarToRow (2) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) + : : : : : : +- CometNativeColumnarToRow (7) : : : : : : +- CometProject (6) : : : : : : +- CometFilter (5) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) + : : : : : +- CometNativeColumnarToRow (14) : : : : : +- CometProject (13) : : : : : +- CometFilter (12) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) + : : : : +- CometNativeColumnarToRow (20) : : : : +- CometFilter (19) : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) + : : : +- CometNativeColumnarToRow (27) : : : +- CometProject (26) : : : +- CometFilter (25) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) : : +- ReusedExchange (31) : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometProject (36) : +- CometFilter (35) : +- CometNativeScan parquet spark_catalog.default.item (34) :- * HashAggregate (73) - : +- * CometColumnarToRow (72) + : +- CometNativeColumnarToRow (72) : +- CometColumnarExchange (71) : +- * HashAggregate (70) : +- * Project (69) @@ -68,14 +68,14 @@ TakeOrderedAndProject (161) : : : : : +- ReusedExchange (51) : : : : +- ReusedExchange (54) : : : +- BroadcastExchange (61) - : : : +- * CometColumnarToRow (60) + : : : +- CometNativeColumnarToRow (60) : : : +- CometProject (59) : : : +- CometFilter (58) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (57) : : +- ReusedExchange (64) : +- ReusedExchange (67) :- * HashAggregate (102) - : +- * CometColumnarToRow (101) + : +- CometNativeColumnarToRow (101) : +- CometColumnarExchange (100) : +- * HashAggregate (99) : +- * Project (98) @@ -97,14 +97,14 @@ TakeOrderedAndProject (161) : : : : : +- ReusedExchange (80) : : : : +- ReusedExchange (83) : : : +- BroadcastExchange (90) - : : : +- * CometColumnarToRow (89) + : : : +- CometNativeColumnarToRow (89) : : : +- CometProject (88) : : : +- CometFilter (87) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (86) : : +- ReusedExchange (93) : +- ReusedExchange (96) :- * HashAggregate (131) - : +- * CometColumnarToRow (130) + : +- CometNativeColumnarToRow (130) : +- CometColumnarExchange (129) : +- * HashAggregate (128) : +- * Project (127) @@ -126,14 +126,14 @@ TakeOrderedAndProject (161) : : : : : +- ReusedExchange (109) : : : : +- ReusedExchange (112) : : : +- BroadcastExchange (119) - : : : +- * CometColumnarToRow (118) + : : : +- CometNativeColumnarToRow (118) : : : +- CometProject (117) : : : +- CometFilter (116) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (115) : : +- ReusedExchange (122) : +- ReusedExchange (125) +- * HashAggregate (159) - +- * CometColumnarToRow (158) + +- CometNativeColumnarToRow (158) +- CometColumnarExchange (157) +- * HashAggregate (156) +- * Project (155) @@ -157,7 +157,7 @@ TakeOrderedAndProject (161) : : +- ReusedExchange (144) : +- ReusedExchange (147) +- BroadcastExchange (153) - +- * CometColumnarToRow (152) + +- CometNativeColumnarToRow (152) +- CometFilter (151) +- CometNativeScan parquet spark_catalog.default.item (150) @@ -170,10 +170,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) @@ -192,20 +192,20 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [cd_demo_sk#11, cd_dep_count#14] (8) BroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 1] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] @@ -224,20 +224,20 @@ Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#1 Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) CometColumnarToRow [codegen id : 2] +(14) CometNativeColumnarToRow Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] (15) BroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 7] +(16) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(17) Project [codegen id : 7] +(17) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] @@ -252,20 +252,20 @@ ReadSchema: struct Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [1]: [cd_demo_sk#20] (21) BroadcastExchange Input [1]: [cd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 1] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -284,33 +284,33 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] -(27) CometColumnarToRow [codegen id : 4] +(27) CometNativeColumnarToRow Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (28) BroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(30) Project [codegen id : 7] +(30) Project [codegen id : 1] Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (31) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#26] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] @@ -329,24 +329,24 @@ Condition : isnotnull(i_item_sk#27) Input [2]: [i_item_sk#27, i_item_id#28] Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] -(37) CometColumnarToRow [codegen id : 6] +(37) CometNativeColumnarToRow Input [2]: [i_item_sk#27, i_item_id#29] (38) BroadcastExchange Input [2]: [i_item_sk#27, i_item_id#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 7] +(39) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(40) Project [codegen id : 7] +(40) Project [codegen id : 1] Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] -(41) HashAggregate [codegen id : 7] +(41) HashAggregate [codegen id : 1] Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] @@ -357,10 +357,10 @@ Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, c Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 8] +(43) CometNativeColumnarToRow Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -(44) HashAggregate [codegen id : 8] +(44) HashAggregate [codegen id : 2] Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] @@ -375,49 +375,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 15] +(46) ColumnarToRow [codegen id : 3] Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -(47) Filter [codegen id : 15] +(47) Filter [codegen id : 3] Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) (48) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#92, cd_dep_count#93] -(49) BroadcastHashJoin [codegen id : 15] +(49) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_cdemo_sk#84] Right keys [1]: [cd_demo_sk#92] Join type: Inner Join condition: None -(50) Project [codegen id : 15] +(50) Project [codegen id : 3] Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93] Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#92, cd_dep_count#93] (51) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] -(52) BroadcastHashJoin [codegen id : 15] +(52) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_customer_sk#83] Right keys [1]: [c_customer_sk#94] Join type: Inner Join condition: None -(53) Project [codegen id : 15] +(53) Project [codegen id : 3] Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] (54) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#98] -(55) BroadcastHashJoin [codegen id : 15] +(55) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#95] Right keys [1]: [cd_demo_sk#98] Join type: Inner Join condition: None -(56) Project [codegen id : 15] +(56) Project [codegen id : 3] Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97] Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97, cd_demo_sk#98] @@ -436,50 +436,50 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] Arguments: [ca_address_sk#99, ca_state#25, ca_country#101], [ca_address_sk#99, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#100, 2, true, false, true) AS ca_state#25, ca_country#101] -(60) CometColumnarToRow [codegen id : 12] +(60) CometNativeColumnarToRow Input [3]: [ca_address_sk#99, ca_state#25, ca_country#101] (61) BroadcastExchange Input [3]: [ca_address_sk#99, ca_state#25, ca_country#101] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(62) BroadcastHashJoin [codegen id : 15] +(62) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#96] Right keys [1]: [ca_address_sk#99] Join type: Inner Join condition: None -(63) Project [codegen id : 15] +(63) Project [codegen id : 3] Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101] Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97, ca_address_sk#99, ca_state#25, ca_country#101] (64) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#102] -(65) BroadcastHashJoin [codegen id : 15] +(65) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#91] Right keys [1]: [d_date_sk#102] Join type: Inner Join condition: None -(66) Project [codegen id : 15] +(66) Project [codegen id : 3] Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101] Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101, d_date_sk#102] (67) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#103, i_item_id#29] -(68) BroadcastHashJoin [codegen id : 15] +(68) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#85] Right keys [1]: [i_item_sk#103] Join type: Inner Join condition: None -(69) Project [codegen id : 15] +(69) Project [codegen id : 3] Output [10]: [i_item_id#29, ca_country#101, ca_state#25, cast(cs_quantity#86 as decimal(12,2)) AS agg1#104, cast(cs_list_price#87 as decimal(12,2)) AS agg2#105, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#106, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#107, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#108, cast(c_birth_year#97 as decimal(12,2)) AS agg6#109, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#110] Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101, i_item_sk#103, i_item_id#29] -(70) HashAggregate [codegen id : 15] +(70) HashAggregate [codegen id : 3] Input [10]: [i_item_id#29, ca_country#101, ca_state#25, agg1#104, agg2#105, agg3#106, agg4#107, agg5#108, agg6#109, agg7#110] Keys [3]: [i_item_id#29, ca_country#101, ca_state#25] Functions [7]: [partial_avg(agg1#104), partial_avg(agg2#105), partial_avg(agg3#106), partial_avg(agg4#107), partial_avg(agg5#108), partial_avg(agg6#109), partial_avg(agg7#110)] @@ -490,10 +490,10 @@ Results [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#125, count#126, su Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138] Arguments: hashpartitioning(i_item_id#29, ca_country#101, ca_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(72) CometColumnarToRow [codegen id : 16] +(72) CometNativeColumnarToRow Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138] -(73) HashAggregate [codegen id : 16] +(73) HashAggregate [codegen id : 4] Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138] Keys [3]: [i_item_id#29, ca_country#101, ca_state#25] Functions [7]: [avg(agg1#104), avg(agg2#105), avg(agg3#106), avg(agg4#107), avg(agg5#108), avg(agg6#109), avg(agg7#110)] @@ -508,49 +508,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#162), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 23] +(75) ColumnarToRow [codegen id : 5] Input [9]: [cs_bill_customer_sk#154, cs_bill_cdemo_sk#155, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162] -(76) Filter [codegen id : 23] +(76) Filter [codegen id : 5] Input [9]: [cs_bill_customer_sk#154, cs_bill_cdemo_sk#155, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162] Condition : ((isnotnull(cs_bill_cdemo_sk#155) AND isnotnull(cs_bill_customer_sk#154)) AND isnotnull(cs_item_sk#156)) (77) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#163, cd_dep_count#164] -(78) BroadcastHashJoin [codegen id : 23] +(78) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_bill_cdemo_sk#155] Right keys [1]: [cd_demo_sk#163] Join type: Inner Join condition: None -(79) Project [codegen id : 23] +(79) Project [codegen id : 5] Output [9]: [cs_bill_customer_sk#154, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164] Input [11]: [cs_bill_customer_sk#154, cs_bill_cdemo_sk#155, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_demo_sk#163, cd_dep_count#164] (80) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#165, c_current_cdemo_sk#166, c_current_addr_sk#167, c_birth_year#168] -(81) BroadcastHashJoin [codegen id : 23] +(81) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_bill_customer_sk#154] Right keys [1]: [c_customer_sk#165] Join type: Inner Join condition: None -(82) Project [codegen id : 23] +(82) Project [codegen id : 5] Output [11]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_current_cdemo_sk#166, c_current_addr_sk#167, c_birth_year#168] Input [13]: [cs_bill_customer_sk#154, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_customer_sk#165, c_current_cdemo_sk#166, c_current_addr_sk#167, c_birth_year#168] (83) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#169] -(84) BroadcastHashJoin [codegen id : 23] +(84) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#166] Right keys [1]: [cd_demo_sk#169] Join type: Inner Join condition: None -(85) Project [codegen id : 23] +(85) Project [codegen id : 5] Output [10]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_current_addr_sk#167, c_birth_year#168] Input [12]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_current_cdemo_sk#166, c_current_addr_sk#167, c_birth_year#168, cd_demo_sk#169] @@ -569,50 +569,50 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [3]: [ca_address_sk#170, ca_state#171, ca_country#172] Arguments: [ca_address_sk#170, ca_country#172], [ca_address_sk#170, ca_country#172] -(89) CometColumnarToRow [codegen id : 20] +(89) CometNativeColumnarToRow Input [2]: [ca_address_sk#170, ca_country#172] (90) BroadcastExchange Input [2]: [ca_address_sk#170, ca_country#172] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(91) BroadcastHashJoin [codegen id : 23] +(91) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#167] Right keys [1]: [ca_address_sk#170] Join type: Inner Join condition: None -(92) Project [codegen id : 23] +(92) Project [codegen id : 5] Output [10]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_birth_year#168, ca_country#172] Input [12]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_current_addr_sk#167, c_birth_year#168, ca_address_sk#170, ca_country#172] (93) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#173] -(94) BroadcastHashJoin [codegen id : 23] +(94) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#162] Right keys [1]: [d_date_sk#173] Join type: Inner Join condition: None -(95) Project [codegen id : 23] +(95) Project [codegen id : 5] Output [9]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cd_dep_count#164, c_birth_year#168, ca_country#172] Input [11]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_birth_year#168, ca_country#172, d_date_sk#173] (96) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#174, i_item_id#29] -(97) BroadcastHashJoin [codegen id : 23] +(97) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#156] Right keys [1]: [i_item_sk#174] Join type: Inner Join condition: None -(98) Project [codegen id : 23] +(98) Project [codegen id : 5] Output [9]: [i_item_id#29, ca_country#172, cast(cs_quantity#157 as decimal(12,2)) AS agg1#175, cast(cs_list_price#158 as decimal(12,2)) AS agg2#176, cast(cs_coupon_amt#160 as decimal(12,2)) AS agg3#177, cast(cs_sales_price#159 as decimal(12,2)) AS agg4#178, cast(cs_net_profit#161 as decimal(12,2)) AS agg5#179, cast(c_birth_year#168 as decimal(12,2)) AS agg6#180, cast(cd_dep_count#164 as decimal(12,2)) AS agg7#181] Input [11]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cd_dep_count#164, c_birth_year#168, ca_country#172, i_item_sk#174, i_item_id#29] -(99) HashAggregate [codegen id : 23] +(99) HashAggregate [codegen id : 5] Input [9]: [i_item_id#29, ca_country#172, agg1#175, agg2#176, agg3#177, agg4#178, agg5#179, agg6#180, agg7#181] Keys [2]: [i_item_id#29, ca_country#172] Functions [7]: [partial_avg(agg1#175), partial_avg(agg2#176), partial_avg(agg3#177), partial_avg(agg4#178), partial_avg(agg5#179), partial_avg(agg6#180), partial_avg(agg7#181)] @@ -623,10 +623,10 @@ Results [16]: [i_item_id#29, ca_country#172, sum#196, count#197, sum#198, count# Input [16]: [i_item_id#29, ca_country#172, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209] Arguments: hashpartitioning(i_item_id#29, ca_country#172, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(101) CometColumnarToRow [codegen id : 24] +(101) CometNativeColumnarToRow Input [16]: [i_item_id#29, ca_country#172, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209] -(102) HashAggregate [codegen id : 24] +(102) HashAggregate [codegen id : 6] Input [16]: [i_item_id#29, ca_country#172, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209] Keys [2]: [i_item_id#29, ca_country#172] Functions [7]: [avg(agg1#175), avg(agg2#176), avg(agg3#177), avg(agg4#178), avg(agg5#179), avg(agg6#180), avg(agg7#181)] @@ -641,49 +641,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#234), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 31] +(104) ColumnarToRow [codegen id : 7] Input [9]: [cs_bill_customer_sk#226, cs_bill_cdemo_sk#227, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234] -(105) Filter [codegen id : 31] +(105) Filter [codegen id : 7] Input [9]: [cs_bill_customer_sk#226, cs_bill_cdemo_sk#227, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234] Condition : ((isnotnull(cs_bill_cdemo_sk#227) AND isnotnull(cs_bill_customer_sk#226)) AND isnotnull(cs_item_sk#228)) (106) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#235, cd_dep_count#236] -(107) BroadcastHashJoin [codegen id : 31] +(107) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_bill_cdemo_sk#227] Right keys [1]: [cd_demo_sk#235] Join type: Inner Join condition: None -(108) Project [codegen id : 31] +(108) Project [codegen id : 7] Output [9]: [cs_bill_customer_sk#226, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236] Input [11]: [cs_bill_customer_sk#226, cs_bill_cdemo_sk#227, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_demo_sk#235, cd_dep_count#236] (109) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#237, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] -(110) BroadcastHashJoin [codegen id : 31] +(110) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_bill_customer_sk#226] Right keys [1]: [c_customer_sk#237] Join type: Inner Join condition: None -(111) Project [codegen id : 31] +(111) Project [codegen id : 7] Output [11]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] Input [13]: [cs_bill_customer_sk#226, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_customer_sk#237, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] (112) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#241] -(113) BroadcastHashJoin [codegen id : 31] +(113) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_current_cdemo_sk#238] Right keys [1]: [cd_demo_sk#241] Join type: Inner Join condition: None -(114) Project [codegen id : 31] +(114) Project [codegen id : 7] Output [10]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_current_addr_sk#239, c_birth_year#240] Input [12]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240, cd_demo_sk#241] @@ -702,50 +702,50 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [ca_address_sk#242, ca_state#243] Arguments: [ca_address_sk#242], [ca_address_sk#242] -(118) CometColumnarToRow [codegen id : 28] +(118) CometNativeColumnarToRow Input [1]: [ca_address_sk#242] (119) BroadcastExchange Input [1]: [ca_address_sk#242] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(120) BroadcastHashJoin [codegen id : 31] +(120) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_current_addr_sk#239] Right keys [1]: [ca_address_sk#242] Join type: Inner Join condition: None -(121) Project [codegen id : 31] +(121) Project [codegen id : 7] Output [9]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_birth_year#240] Input [11]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_current_addr_sk#239, c_birth_year#240, ca_address_sk#242] (122) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#244] -(123) BroadcastHashJoin [codegen id : 31] +(123) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#234] Right keys [1]: [d_date_sk#244] Join type: Inner Join condition: None -(124) Project [codegen id : 31] +(124) Project [codegen id : 7] Output [8]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cd_dep_count#236, c_birth_year#240] Input [10]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_birth_year#240, d_date_sk#244] (125) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#245, i_item_id#29] -(126) BroadcastHashJoin [codegen id : 31] +(126) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_item_sk#228] Right keys [1]: [i_item_sk#245] Join type: Inner Join condition: None -(127) Project [codegen id : 31] +(127) Project [codegen id : 7] Output [8]: [i_item_id#29, cast(cs_quantity#229 as decimal(12,2)) AS agg1#246, cast(cs_list_price#230 as decimal(12,2)) AS agg2#247, cast(cs_coupon_amt#232 as decimal(12,2)) AS agg3#248, cast(cs_sales_price#231 as decimal(12,2)) AS agg4#249, cast(cs_net_profit#233 as decimal(12,2)) AS agg5#250, cast(c_birth_year#240 as decimal(12,2)) AS agg6#251, cast(cd_dep_count#236 as decimal(12,2)) AS agg7#252] Input [10]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cd_dep_count#236, c_birth_year#240, i_item_sk#245, i_item_id#29] -(128) HashAggregate [codegen id : 31] +(128) HashAggregate [codegen id : 7] Input [8]: [i_item_id#29, agg1#246, agg2#247, agg3#248, agg4#249, agg5#250, agg6#251, agg7#252] Keys [1]: [i_item_id#29] Functions [7]: [partial_avg(agg1#246), partial_avg(agg2#247), partial_avg(agg3#248), partial_avg(agg4#249), partial_avg(agg5#250), partial_avg(agg6#251), partial_avg(agg7#252)] @@ -756,10 +756,10 @@ Results [15]: [i_item_id#29, sum#267, count#268, sum#269, count#270, sum#271, co Input [15]: [i_item_id#29, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(130) CometColumnarToRow [codegen id : 32] +(130) CometNativeColumnarToRow Input [15]: [i_item_id#29, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] -(131) HashAggregate [codegen id : 32] +(131) HashAggregate [codegen id : 8] Input [15]: [i_item_id#29, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] Keys [1]: [i_item_id#29] Functions [7]: [avg(agg1#246), avg(agg2#247), avg(agg3#248), avg(agg4#249), avg(agg5#250), avg(agg6#251), avg(agg7#252)] @@ -774,75 +774,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#306), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 39] +(133) ColumnarToRow [codegen id : 9] Input [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] -(134) Filter [codegen id : 39] +(134) Filter [codegen id : 9] Input [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] Condition : ((isnotnull(cs_bill_cdemo_sk#299) AND isnotnull(cs_bill_customer_sk#298)) AND isnotnull(cs_item_sk#300)) (135) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#307, cd_dep_count#308] -(136) BroadcastHashJoin [codegen id : 39] +(136) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_bill_cdemo_sk#299] Right keys [1]: [cd_demo_sk#307] Join type: Inner Join condition: None -(137) Project [codegen id : 39] +(137) Project [codegen id : 9] Output [9]: [cs_bill_customer_sk#298, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308] Input [11]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_demo_sk#307, cd_dep_count#308] (138) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#309, c_current_cdemo_sk#310, c_current_addr_sk#311, c_birth_year#312] -(139) BroadcastHashJoin [codegen id : 39] +(139) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_bill_customer_sk#298] Right keys [1]: [c_customer_sk#309] Join type: Inner Join condition: None -(140) Project [codegen id : 39] +(140) Project [codegen id : 9] Output [11]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_current_cdemo_sk#310, c_current_addr_sk#311, c_birth_year#312] Input [13]: [cs_bill_customer_sk#298, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_customer_sk#309, c_current_cdemo_sk#310, c_current_addr_sk#311, c_birth_year#312] (141) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#313] -(142) BroadcastHashJoin [codegen id : 39] +(142) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#310] Right keys [1]: [cd_demo_sk#313] Join type: Inner Join condition: None -(143) Project [codegen id : 39] +(143) Project [codegen id : 9] Output [10]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_current_addr_sk#311, c_birth_year#312] Input [12]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_current_cdemo_sk#310, c_current_addr_sk#311, c_birth_year#312, cd_demo_sk#313] (144) ReusedExchange [Reuses operator id: 119] Output [1]: [ca_address_sk#314] -(145) BroadcastHashJoin [codegen id : 39] +(145) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#311] Right keys [1]: [ca_address_sk#314] Join type: Inner Join condition: None -(146) Project [codegen id : 39] +(146) Project [codegen id : 9] Output [9]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_birth_year#312] Input [11]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_current_addr_sk#311, c_birth_year#312, ca_address_sk#314] (147) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#315] -(148) BroadcastHashJoin [codegen id : 39] +(148) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#306] Right keys [1]: [d_date_sk#315] Join type: Inner Join condition: None -(149) Project [codegen id : 39] +(149) Project [codegen id : 9] Output [8]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cd_dep_count#308, c_birth_year#312] Input [10]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_birth_year#312, d_date_sk#315] @@ -857,24 +857,24 @@ ReadSchema: struct Input [1]: [i_item_sk#316] Condition : isnotnull(i_item_sk#316) -(152) CometColumnarToRow [codegen id : 38] +(152) CometNativeColumnarToRow Input [1]: [i_item_sk#316] (153) BroadcastExchange Input [1]: [i_item_sk#316] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(154) BroadcastHashJoin [codegen id : 39] +(154) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_item_sk#300] Right keys [1]: [i_item_sk#316] Join type: Inner Join condition: None -(155) Project [codegen id : 39] +(155) Project [codegen id : 9] Output [7]: [cast(cs_quantity#301 as decimal(12,2)) AS agg1#317, cast(cs_list_price#302 as decimal(12,2)) AS agg2#318, cast(cs_coupon_amt#304 as decimal(12,2)) AS agg3#319, cast(cs_sales_price#303 as decimal(12,2)) AS agg4#320, cast(cs_net_profit#305 as decimal(12,2)) AS agg5#321, cast(c_birth_year#312 as decimal(12,2)) AS agg6#322, cast(cd_dep_count#308 as decimal(12,2)) AS agg7#323] Input [9]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cd_dep_count#308, c_birth_year#312, i_item_sk#316] -(156) HashAggregate [codegen id : 39] +(156) HashAggregate [codegen id : 9] Input [7]: [agg1#317, agg2#318, agg3#319, agg4#320, agg5#321, agg6#322, agg7#323] Keys: [] Functions [7]: [partial_avg(agg1#317), partial_avg(agg2#318), partial_avg(agg3#319), partial_avg(agg4#320), partial_avg(agg5#321), partial_avg(agg6#322), partial_avg(agg7#323)] @@ -885,10 +885,10 @@ Results [14]: [sum#338, count#339, sum#340, count#341, sum#342, count#343, sum#3 Input [14]: [sum#338, count#339, sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(158) CometColumnarToRow [codegen id : 40] +(158) CometNativeColumnarToRow Input [14]: [sum#338, count#339, sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351] -(159) HashAggregate [codegen id : 40] +(159) HashAggregate [codegen id : 10] Input [14]: [sum#338, count#339, sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351] Keys: [] Functions [7]: [avg(agg1#317), avg(agg2#318), avg(agg3#319), avg(agg4#320), avg(agg5#321), avg(agg6#322), avg(agg7#323)] @@ -905,7 +905,7 @@ Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (166) -+- * CometColumnarToRow (165) ++- CometNativeColumnarToRow (165) +- CometProject (164) +- CometFilter (163) +- CometNativeScan parquet spark_catalog.default.date_dim (162) @@ -926,7 +926,7 @@ Condition : ((isnotnull(d_year#370) AND (d_year#370 = 2001)) AND isnotnull(d_dat Input [2]: [d_date_sk#26, d_year#370] Arguments: [d_date_sk#26], [d_date_sk#26] -(165) CometColumnarToRow [codegen id : 1] +(165) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (166) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt index eb411c16e8..8ad4e890ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,41 +21,41 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -75,36 +75,36 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -124,36 +124,36 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -173,36 +173,36 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -222,31 +222,31 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/simplified.txt index 71745258b5..a123d3d8ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] Union - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -25,59 +25,47 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeColumnarToRow + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeColumnarToRow + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometNativeColumnarToRow + CometFilter [cd_demo_sk] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) + WholeStageCodegen (3) HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -104,22 +92,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #9 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) + WholeStageCodegen (6) HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) + WholeStageCodegen (5) HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -146,22 +132,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #11 - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #12 - WholeStageCodegen (31) + WholeStageCodegen (7) HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -188,22 +172,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #13 - WholeStageCodegen (28) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) + WholeStageCodegen (10) HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #14 - WholeStageCodegen (39) + WholeStageCodegen (9) HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -234,8 +216,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #15 - WholeStageCodegen (38) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt index 72e9bd1b4a..99c06efb48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt @@ -807,7 +807,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, a Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (151) -+- * CometColumnarToRow (150) ++- CometNativeColumnarToRow (150) +- CometProject (149) +- CometFilter (148) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) @@ -828,7 +828,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(150) CometColumnarToRow [codegen id : 1] +(150) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (151) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt index b18a444bb9..148f6d4a6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt index 1572a2a240..ce665c5294 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt index 72e9bd1b4a..99c06efb48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt @@ -807,7 +807,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, a Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (151) -+- * CometColumnarToRow (150) ++- CometNativeColumnarToRow (150) +- CometProject (149) +- CometFilter (148) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) @@ -828,7 +828,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(150) CometColumnarToRow [codegen id : 1] +(150) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (151) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt index b18a444bb9..148f6d4a6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 1572a2a240..ce665c5294 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 @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/explain.txt index 5a32d4019b..460a888704 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.catalog_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) @@ -54,37 +54,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt index 369ec68bb4..6634edcbed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/simplified.txt index 7bc0779e53..0c80fb3016 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt index 322bbe63b3..fb3e9cf73a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt index cf18e68a3d..7d1b101ff1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt index 322bbe63b3..fb3e9cf73a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 cf18e68a3d..7d1b101ff1 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q22.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/explain.txt index aa06ae2c5f..e098401edc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Expand (19) @@ -16,12 +16,12 @@ TakeOrderedAndProject (24) : : : +- Scan parquet spark_catalog.default.inventory (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) +- BroadcastExchange (16) - +- * CometColumnarToRow (15) + +- CometNativeColumnarToRow (15) +- CometNativeScan parquet spark_catalog.default.warehouse (14) @@ -33,23 +33,23 @@ PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] Condition : isnotnull(inv_item_sk#1) (4) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] @@ -68,20 +68,20 @@ Condition : isnotnull(i_item_sk#6) Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#7, 50, true, false, true) AS i_brand#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#10, 50, true, false, true) AS i_product_name#14] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] (11) BroadcastExchange Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] @@ -91,26 +91,26 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(15) CometColumnarToRow [codegen id : 3] +(15) CometNativeColumnarToRow Input: [] (16) BroadcastExchange Input: [] Arguments: IdentityBroadcastMode, [plan_id=2] -(17) BroadcastNestedLoopJoin [codegen id : 4] +(17) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -(19) Expand [codegen id : 4] +(19) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] @@ -121,10 +121,10 @@ Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_gr Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [avg(inv_quantity_on_hand#2)] @@ -139,7 +139,7 @@ Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -160,7 +160,7 @@ Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#26] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt index fb0e21025e..108e3c13f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -15,22 +15,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/simplified.txt index 457dff46e2..954f5cb42e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,25 +19,19 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometNativeScan parquet spark_catalog.default.warehouse + CometNativeColumnarToRow + CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt index f85c898208..daa4c97503 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject (26) : +- CometFilter (10) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) @@ -95,7 +95,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(15) CometColumnarToRow [codegen id : 2] +(15) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse @@ -104,26 +104,26 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(17) CometColumnarToRow [codegen id : 1] +(17) CometNativeColumnarToRow Input: [] (18) BroadcastExchange Input: [] Arguments: IdentityBroadcastMode, [plan_id=1] -(19) BroadcastNestedLoopJoin [codegen id : 2] +(19) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(20) Project [codegen id : 2] +(20) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(21) Expand [codegen id : 2] +(21) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] @@ -134,10 +134,10 @@ Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_gr Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(24) CometColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(25) HashAggregate [codegen id : 3] +(25) HashAggregate [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [avg(inv_quantity_on_hand#2)] @@ -152,7 +152,7 @@ Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt index 6cb89465d2..5da4848ebe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -28,7 +28,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt index 221c6063ce..11d581a487 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,12 +19,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -35,7 +33,5 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + CometNativeColumnarToRow + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt index f85c898208..daa4c97503 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject (26) : +- CometFilter (10) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) @@ -95,7 +95,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(15) CometColumnarToRow [codegen id : 2] +(15) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse @@ -104,26 +104,26 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(17) CometColumnarToRow [codegen id : 1] +(17) CometNativeColumnarToRow Input: [] (18) BroadcastExchange Input: [] Arguments: IdentityBroadcastMode, [plan_id=1] -(19) BroadcastNestedLoopJoin [codegen id : 2] +(19) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(20) Project [codegen id : 2] +(20) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(21) Expand [codegen id : 2] +(21) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] @@ -134,10 +134,10 @@ Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_gr Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(24) CometColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(25) HashAggregate [codegen id : 3] +(25) HashAggregate [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [avg(inv_quantity_on_hand#2)] @@ -152,7 +152,7 @@ Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt index 6cb89465d2..5da4848ebe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -28,7 +28,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt index 221c6063ce..11d581a487 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,12 +19,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -35,7 +33,5 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + CometNativeColumnarToRow + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/explain.txt index 0998977eb2..b83f1239c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/explain.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject (55) :- * HashAggregate (25) : +- * HashAggregate (24) : +- * HashAggregate (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometColumnarExchange (21) : +- * HashAggregate (20) : +- * Project (19) @@ -18,41 +18,41 @@ TakeOrderedAndProject (55) : : : : +- Scan parquet spark_catalog.default.inventory (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.item (7) : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.warehouse (14) :- * HashAggregate (32) - : +- * CometColumnarToRow (31) + : +- CometNativeColumnarToRow (31) : +- CometColumnarExchange (30) : +- * HashAggregate (29) : +- * HashAggregate (28) - : +- * CometColumnarToRow (27) + : +- CometNativeColumnarToRow (27) : +- ReusedExchange (26) :- * HashAggregate (39) - : +- * CometColumnarToRow (38) + : +- CometNativeColumnarToRow (38) : +- CometColumnarExchange (37) : +- * HashAggregate (36) : +- * HashAggregate (35) - : +- * CometColumnarToRow (34) + : +- CometNativeColumnarToRow (34) : +- ReusedExchange (33) :- * HashAggregate (46) - : +- * CometColumnarToRow (45) + : +- CometNativeColumnarToRow (45) : +- CometColumnarExchange (44) : +- * HashAggregate (43) : +- * HashAggregate (42) - : +- * CometColumnarToRow (41) + : +- CometNativeColumnarToRow (41) : +- ReusedExchange (40) +- * HashAggregate (53) - +- * CometColumnarToRow (52) + +- CometNativeColumnarToRow (52) +- CometColumnarExchange (51) +- * HashAggregate (50) +- * HashAggregate (49) - +- * CometColumnarToRow (48) + +- CometNativeColumnarToRow (48) +- ReusedExchange (47) @@ -64,23 +64,23 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (4) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] @@ -99,20 +99,20 @@ Condition : isnotnull(i_item_sk#7) Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] (11) BroadcastExchange Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] @@ -127,24 +127,24 @@ ReadSchema: struct Input [1]: [w_warehouse_sk#16] Condition : isnotnull(w_warehouse_sk#16) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [w_warehouse_sk#16] (17) BroadcastExchange Input [1]: [w_warehouse_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#16] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] @@ -155,24 +155,24 @@ Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#3)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] Results [5]: [i_product_name#15 AS i_product_name#22, i_brand#12 AS i_brand#23, i_class#13 AS i_class#24, i_category#14 AS i_category#25, avg(inv_quantity_on_hand#3)#21 AS qoh#26] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#26] Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] Functions [1]: [partial_avg(qoh#26)] Aggregate Attributes [2]: [sum#27, count#28] Results [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 2] Input [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] Functions [1]: [avg(qoh#26)] @@ -182,17 +182,17 @@ Results [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, avg(qoh# (26) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] -(27) CometColumnarToRow [codegen id : 10] +(27) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] -(28) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 4] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#35)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#21] Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(inv_quantity_on_hand#35)#21 AS qoh#36] -(29) HashAggregate [codegen id : 10] +(29) HashAggregate [codegen id : 4] Input [4]: [i_product_name#15, i_brand#12, i_class#13, qoh#36] Keys [3]: [i_product_name#15, i_brand#12, i_class#13] Functions [1]: [partial_avg(qoh#36)] @@ -203,10 +203,10 @@ Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(31) CometColumnarToRow [codegen id : 11] +(31) CometNativeColumnarToRow Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -(32) HashAggregate [codegen id : 11] +(32) HashAggregate [codegen id : 5] Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] Keys [3]: [i_product_name#15, i_brand#12, i_class#13] Functions [1]: [avg(qoh#36)] @@ -216,17 +216,17 @@ Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#42, (33) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] -(34) CometColumnarToRow [codegen id : 16] +(34) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] -(35) HashAggregate [codegen id : 16] +(35) HashAggregate [codegen id : 7] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#46)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#21] Results [3]: [i_product_name#15, i_brand#12, avg(inv_quantity_on_hand#46)#21 AS qoh#47] -(36) HashAggregate [codegen id : 16] +(36) HashAggregate [codegen id : 7] Input [3]: [i_product_name#15, i_brand#12, qoh#47] Keys [2]: [i_product_name#15, i_brand#12] Functions [1]: [partial_avg(qoh#47)] @@ -237,10 +237,10 @@ Results [4]: [i_product_name#15, i_brand#12, sum#50, count#51] Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(38) CometColumnarToRow [codegen id : 17] +(38) CometNativeColumnarToRow Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] -(39) HashAggregate [codegen id : 17] +(39) HashAggregate [codegen id : 8] Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] Keys [2]: [i_product_name#15, i_brand#12] Functions [1]: [avg(qoh#47)] @@ -250,17 +250,17 @@ Results [5]: [i_product_name#15, i_brand#12, null AS i_class#53, null AS i_categ (40) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] -(41) CometColumnarToRow [codegen id : 22] +(41) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] -(42) HashAggregate [codegen id : 22] +(42) HashAggregate [codegen id : 10] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#58)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#58)#21] Results [2]: [i_product_name#15, avg(inv_quantity_on_hand#58)#21 AS qoh#59] -(43) HashAggregate [codegen id : 22] +(43) HashAggregate [codegen id : 10] Input [2]: [i_product_name#15, qoh#59] Keys [1]: [i_product_name#15] Functions [1]: [partial_avg(qoh#59)] @@ -271,10 +271,10 @@ Results [3]: [i_product_name#15, sum#62, count#63] Input [3]: [i_product_name#15, sum#62, count#63] Arguments: hashpartitioning(i_product_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(45) CometColumnarToRow [codegen id : 23] +(45) CometNativeColumnarToRow Input [3]: [i_product_name#15, sum#62, count#63] -(46) HashAggregate [codegen id : 23] +(46) HashAggregate [codegen id : 11] Input [3]: [i_product_name#15, sum#62, count#63] Keys [1]: [i_product_name#15] Functions [1]: [avg(qoh#59)] @@ -284,17 +284,17 @@ Results [5]: [i_product_name#15, null AS i_brand#65, null AS i_class#66, null AS (47) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] -(48) CometColumnarToRow [codegen id : 28] +(48) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] -(49) HashAggregate [codegen id : 28] +(49) HashAggregate [codegen id : 13] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#71)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#71)#21] Results [1]: [avg(inv_quantity_on_hand#71)#21 AS qoh#72] -(50) HashAggregate [codegen id : 28] +(50) HashAggregate [codegen id : 13] Input [1]: [qoh#72] Keys: [] Functions [1]: [partial_avg(qoh#72)] @@ -305,10 +305,10 @@ Results [2]: [sum#75, count#76] Input [2]: [sum#75, count#76] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(52) CometColumnarToRow [codegen id : 29] +(52) CometNativeColumnarToRow Input [2]: [sum#75, count#76] -(53) HashAggregate [codegen id : 29] +(53) HashAggregate [codegen id : 14] Input [2]: [sum#75, count#76] Keys: [] Functions [1]: [avg(qoh#72)] @@ -325,7 +325,7 @@ Arguments: 100, [qoh#32 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (60) -+- * CometColumnarToRow (59) ++- CometNativeColumnarToRow (59) +- CometProject (58) +- CometFilter (57) +- CometNativeScan parquet spark_catalog.default.date_dim (56) @@ -346,7 +346,7 @@ Condition : (((isnotnull(d_month_seq#83) AND (d_month_seq#83 >= 1212)) AND (d_mo Input [2]: [d_date_sk#6, d_month_seq#83] Arguments: [d_date_sk#6], [d_date_sk#6] -(59) CometColumnarToRow [codegen id : 1] +(59) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (60) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt index 543f666a74..eaa5438d91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject :- HashAggregate : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -17,30 +17,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -54,30 +54,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -91,30 +91,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -128,30 +128,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -165,22 +165,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/simplified.txt index a55feaac8c..4f9ec6fb7a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/simplified.txt @@ -1,13 +1,13 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] @@ -21,70 +21,64 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) + CometNativeColumnarToRow + CometFilter [w_warehouse_sk] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + WholeStageCodegen (5) HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) + WholeStageCodegen (4) HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) + WholeStageCodegen (11) HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name] #7 - WholeStageCodegen (22) + WholeStageCodegen (10) HashAggregate [i_product_name,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) + WholeStageCodegen (14) HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #8 - WholeStageCodegen (28) + WholeStageCodegen (13) HashAggregate [qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow 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/q22a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt index bf16cc4d21..e951ab35c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt @@ -270,7 +270,7 @@ Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt index c018673888..4ccef655b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt index b59605103e..e6db35bb18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt index bf16cc4d21..e951ab35c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt @@ -270,7 +270,7 @@ Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt index c018673888..4ccef655b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 b59605103e..e6db35bb18 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/explain.txt index 696a4579e6..321a81da13 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/explain.txt @@ -1,19 +1,19 @@ == Physical Plan == -* CometColumnarToRow (50) +CometNativeColumnarToRow (50) +- CometSort (49) +- CometColumnarExchange (48) +- * Filter (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) + :- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometProject (24) @@ -45,7 +45,7 @@ : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_address (32) @@ -190,7 +190,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 2] +(31) CometNativeColumnarToRow Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] (32) CometNativeScan parquet spark_catalog.default.customer_address @@ -208,24 +208,24 @@ Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isno Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] (36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -236,17 +236,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometNativeColumnarToRow Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] @@ -257,17 +257,17 @@ Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty# Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometNativeColumnarToRow Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) @@ -279,23 +279,23 @@ Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(50) CometNativeColumnarToRow Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] * HashAggregate (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometColumnarExchange (78) +- * HashAggregate (77) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- CometColumnarExchange (74) +- * HashAggregate (73) +- * Project (72) +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) + :- CometNativeColumnarToRow (69) : +- CometProject (68) : +- CometBroadcastHashJoin (67) : :- CometProject (65) @@ -392,23 +392,23 @@ Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(69) CometColumnarToRow [codegen id : 2] +(69) CometNativeColumnarToRow Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] (70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(71) BroadcastHashJoin [codegen id : 2] +(71) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(72) Project [codegen id : 1] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(73) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] @@ -419,17 +419,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometColumnarToRow [codegen id : 3] +(75) CometNativeColumnarToRow Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(76) HashAggregate [codegen id : 3] +(76) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] -(77) HashAggregate [codegen id : 3] +(77) HashAggregate [codegen id : 2] Input [1]: [netpaid#73] Keys: [] Functions [1]: [partial_avg(netpaid#73)] @@ -440,10 +440,10 @@ Results [2]: [sum#76, count#77] Input [2]: [sum#76, count#77] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 4] +(79) CometNativeColumnarToRow Input [2]: [sum#76, count#77] -(80) HashAggregate [codegen id : 4] +(80) HashAggregate [codegen id : 3] Input [2]: [sum#76, count#77] Keys: [] Functions [1]: [avg(netpaid#73)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/extended.txt index 53a4c64a61..4c22965f0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/extended.txt @@ -1,19 +1,19 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Filter : +- Subquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometProject @@ -45,21 +45,21 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometProject @@ -91,7 +91,7 @@ CometColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/simplified.txt index b24e51723a..4758db93c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/simplified.txt @@ -1,64 +1,26 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] +CometNativeColumnarToRow + CometSort [c_last_name,c_first_name,s_store_name,paid] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (3) + Filter [paid] + Subquery #1 + WholeStageCodegen (3) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -68,32 +30,66 @@ WholeStageCodegen (5) CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + ReusedExchange [sr_item_sk,sr_ticket_number] #5 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + WholeStageCodegen (1) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + InputAdapter + CometNativeColumnarToRow + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometExchange [ss_ticket_number,ss_item_sk] #4 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometSort [sr_item_sk,sr_ticket_number] + CometExchange [sr_ticket_number,sr_item_sk] #5 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #9 + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt index 17ef709df5..b98174ffdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt @@ -45,7 +45,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -190,7 +190,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -208,24 +208,24 @@ Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isno Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] (36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -236,17 +236,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] @@ -257,17 +257,17 @@ Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty# Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) @@ -279,7 +279,7 @@ Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(50) CometColumnarToRow [codegen id : 4] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== @@ -392,23 +392,23 @@ Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(69) CometColumnarToRow [codegen id : 2] +(69) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] (70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(71) BroadcastHashJoin [codegen id : 2] +(71) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(72) Project [codegen id : 1] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(73) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] @@ -419,17 +419,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometColumnarToRow [codegen id : 3] +(75) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(76) HashAggregate [codegen id : 3] +(76) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] -(77) HashAggregate [codegen id : 3] +(77) HashAggregate [codegen id : 2] Input [1]: [netpaid#73] Keys: [] Functions [1]: [partial_avg(netpaid#73)] @@ -440,10 +440,10 @@ Results [2]: [sum#76, count#77] Input [2]: [sum#76, count#77] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 4] +(79) CometColumnarToRow [codegen id : 3] Input [2]: [sum#76, count#77] -(80) HashAggregate [codegen id : 4] +(80) HashAggregate [codegen id : 3] Input [2]: [sum#76, count#77] Keys: [] Functions [1]: [avg(netpaid#73)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt index b9384ca04f..91b709aa47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt @@ -45,7 +45,7 @@ CometColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -91,7 +91,7 @@ CometColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt index 62f492f632..b657d3dae1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (5) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -47,13 +47,13 @@ WholeStageCodegen (5) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -91,9 +91,7 @@ WholeStageCodegen (5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt index 17ef709df5..b98174ffdb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt @@ -45,7 +45,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -190,7 +190,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -208,24 +208,24 @@ Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isno Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] (36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -236,17 +236,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] @@ -257,17 +257,17 @@ Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty# Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) @@ -279,7 +279,7 @@ Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(50) CometColumnarToRow [codegen id : 4] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== @@ -392,23 +392,23 @@ Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(69) CometColumnarToRow [codegen id : 2] +(69) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] (70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(71) BroadcastHashJoin [codegen id : 2] +(71) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(72) Project [codegen id : 1] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(73) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] @@ -419,17 +419,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometColumnarToRow [codegen id : 3] +(75) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(76) HashAggregate [codegen id : 3] +(76) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] -(77) HashAggregate [codegen id : 3] +(77) HashAggregate [codegen id : 2] Input [1]: [netpaid#73] Keys: [] Functions [1]: [partial_avg(netpaid#73)] @@ -440,10 +440,10 @@ Results [2]: [sum#76, count#77] Input [2]: [sum#76, count#77] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 4] +(79) CometColumnarToRow [codegen id : 3] Input [2]: [sum#76, count#77] -(80) HashAggregate [codegen id : 4] +(80) HashAggregate [codegen id : 3] Input [2]: [sum#76, count#77] Keys: [] Functions [1]: [avg(netpaid#73)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt index b9384ca04f..91b709aa47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt @@ -45,7 +45,7 @@ CometColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -91,7 +91,7 @@ CometColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt index 62f492f632..b657d3dae1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (5) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -47,13 +47,13 @@ WholeStageCodegen (5) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -91,9 +91,7 @@ WholeStageCodegen (5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/explain.txt index c7e92e4c27..f80b4965b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (78) +- Union (77) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * Project (27) @@ -17,23 +17,23 @@ TakeOrderedAndProject (78) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : : +- ReusedExchange (11) : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) + : : +- CometNativeColumnarToRow (17) : : +- CometProject (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.store (14) : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) + : +- CometNativeColumnarToRow (24) : +- CometProject (23) : +- CometFilter (22) : +- CometNativeScan parquet spark_catalog.default.item (21) :- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) @@ -50,13 +50,13 @@ TakeOrderedAndProject (78) : : : : +- ReusedExchange (35) : : : +- ReusedExchange (38) : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) + : : +- CometNativeColumnarToRow (44) : : +- CometProject (43) : : +- CometFilter (42) : : +- CometNativeScan parquet spark_catalog.default.store (41) : +- ReusedExchange (48) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- CometColumnarExchange (74) +- * HashAggregate (73) +- * Project (72) @@ -74,7 +74,7 @@ TakeOrderedAndProject (78) : : +- ReusedExchange (61) : +- ReusedExchange (64) +- BroadcastExchange (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometFilter (68) +- CometNativeScan parquet spark_catalog.default.item (67) @@ -87,10 +87,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) @@ -109,33 +109,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -154,20 +154,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#15, s_state#16] Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [s_store_sk#15, s_state#17] (18) BroadcastExchange Input [2]: [s_store_sk#15, s_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] @@ -186,24 +186,24 @@ Condition : isnotnull(i_item_sk#18) Input [2]: [i_item_sk#18, i_item_id#19] Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [2]: [i_item_sk#18, i_item_id#20] (25) BroadcastExchange Input [2]: [i_item_sk#18, i_item_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] Keys [2]: [i_item_id#21, s_state#22] Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] @@ -214,10 +214,10 @@ Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] Keys [2]: [i_item_id#21, s_state#22] Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] @@ -232,36 +232,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 11] +(33) ColumnarToRow [codegen id : 3] Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -(34) Filter [codegen id : 11] +(34) Filter [codegen id : 3] Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) (35) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#60] -(36) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_cdemo_sk#53] Right keys [1]: [cd_demo_sk#60] Join type: Inner Join condition: None -(37) Project [codegen id : 11] +(37) Project [codegen id : 3] Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#60] (38) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#61] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#59] Right keys [1]: [d_date_sk#61] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 3] Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#61] @@ -280,37 +280,37 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#62, s_state#63] Arguments: [s_store_sk#62], [s_store_sk#62] -(44) CometColumnarToRow [codegen id : 9] +(44) CometNativeColumnarToRow Input [1]: [s_store_sk#62] (45) BroadcastExchange Input [1]: [s_store_sk#62] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(46) BroadcastHashJoin [codegen id : 11] +(46) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#54] Right keys [1]: [s_store_sk#62] Join type: Inner Join condition: None -(47) Project [codegen id : 11] +(47) Project [codegen id : 3] Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#62] (48) ReusedExchange [Reuses operator id: 25] Output [2]: [i_item_sk#64, i_item_id#20] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#52] Right keys [1]: [i_item_sk#64] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 3] Output [5]: [i_item_id#20, ss_quantity#55 AS agg1#65, ss_list_price#56 AS agg2#66, ss_coupon_amt#58 AS agg3#67, ss_sales_price#57 AS agg4#68] Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#64, i_item_id#20] -(51) HashAggregate [codegen id : 11] +(51) HashAggregate [codegen id : 3] Input [5]: [i_item_id#20, agg1#65, agg2#66, agg3#67, agg4#68] Keys [1]: [i_item_id#20] Functions [4]: [partial_avg(agg1#65), partial_avg(UnscaledValue(agg2#66)), partial_avg(UnscaledValue(agg3#67)), partial_avg(UnscaledValue(agg4#68))] @@ -321,10 +321,10 @@ Results [9]: [i_item_id#20, sum#77, count#78, sum#79, count#80, sum#81, count#82 Input [9]: [i_item_id#20, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(53) CometColumnarToRow [codegen id : 12] +(53) CometNativeColumnarToRow Input [9]: [i_item_id#20, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -(54) HashAggregate [codegen id : 12] +(54) HashAggregate [codegen id : 4] Input [9]: [i_item_id#20, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] Keys [1]: [i_item_id#20] Functions [4]: [avg(agg1#65), avg(UnscaledValue(agg2#66)), avg(UnscaledValue(agg3#67)), avg(UnscaledValue(agg4#68))] @@ -339,49 +339,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_s PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 17] +(56) ColumnarToRow [codegen id : 5] Input [8]: [ss_item_sk#95, ss_cdemo_sk#96, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102] -(57) Filter [codegen id : 17] +(57) Filter [codegen id : 5] Input [8]: [ss_item_sk#95, ss_cdemo_sk#96, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102] Condition : ((isnotnull(ss_cdemo_sk#96) AND isnotnull(ss_store_sk#97)) AND isnotnull(ss_item_sk#95)) (58) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#103] -(59) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#96] Right keys [1]: [cd_demo_sk#103] Join type: Inner Join condition: None -(60) Project [codegen id : 17] +(60) Project [codegen id : 5] Output [7]: [ss_item_sk#95, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102] Input [9]: [ss_item_sk#95, ss_cdemo_sk#96, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102, cd_demo_sk#103] (61) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#104] -(62) BroadcastHashJoin [codegen id : 17] +(62) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#102] Right keys [1]: [d_date_sk#104] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(63) Project [codegen id : 5] Output [6]: [ss_item_sk#95, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101] Input [8]: [ss_item_sk#95, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102, d_date_sk#104] (64) ReusedExchange [Reuses operator id: 45] Output [1]: [s_store_sk#105] -(65) BroadcastHashJoin [codegen id : 17] +(65) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#97] Right keys [1]: [s_store_sk#105] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(66) Project [codegen id : 5] Output [5]: [ss_item_sk#95, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101] Input [7]: [ss_item_sk#95, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, s_store_sk#105] @@ -396,24 +396,24 @@ ReadSchema: struct Input [1]: [i_item_sk#106] Condition : isnotnull(i_item_sk#106) -(69) CometColumnarToRow [codegen id : 16] +(69) CometNativeColumnarToRow Input [1]: [i_item_sk#106] (70) BroadcastExchange Input [1]: [i_item_sk#106] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(71) BroadcastHashJoin [codegen id : 17] +(71) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#95] Right keys [1]: [i_item_sk#106] Join type: Inner Join condition: None -(72) Project [codegen id : 17] +(72) Project [codegen id : 5] Output [4]: [ss_quantity#98 AS agg1#107, ss_list_price#99 AS agg2#108, ss_coupon_amt#101 AS agg3#109, ss_sales_price#100 AS agg4#110] Input [6]: [ss_item_sk#95, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, i_item_sk#106] -(73) HashAggregate [codegen id : 17] +(73) HashAggregate [codegen id : 5] Input [4]: [agg1#107, agg2#108, agg3#109, agg4#110] Keys: [] Functions [4]: [partial_avg(agg1#107), partial_avg(UnscaledValue(agg2#108)), partial_avg(UnscaledValue(agg3#109)), partial_avg(UnscaledValue(agg4#110))] @@ -424,10 +424,10 @@ Results [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#12 Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(75) CometColumnarToRow [codegen id : 18] +(75) CometNativeColumnarToRow Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] -(76) HashAggregate [codegen id : 18] +(76) HashAggregate [codegen id : 6] Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] Keys: [] Functions [4]: [avg(agg1#107), avg(UnscaledValue(agg2#108)), avg(UnscaledValue(agg3#109)), avg(UnscaledValue(agg4#110))] @@ -444,7 +444,7 @@ Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometNativeScan parquet spark_catalog.default.date_dim (79) @@ -465,7 +465,7 @@ Condition : ((isnotnull(d_year#138) AND (d_year#138 = 1998)) AND isnotnull(d_dat Input [2]: [d_date_sk#14, d_year#138] Arguments: [d_date_sk#14], [d_date_sk#14] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt index 9926d14ae2..4eec8b0953 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -17,32 +17,32 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,27 +58,27 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -94,22 +94,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/simplified.txt index c7093301af..2dad05ae3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,s_state] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -21,44 +21,36 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (11) + WholeStageCodegen (3) HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -79,20 +71,18 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) + WholeStageCodegen (6) HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #8 - WholeStageCodegen (17) + WholeStageCodegen (5) HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -115,8 +105,6 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ReusedExchange [s_store_sk] #7 InputAdapter BroadcastExchange #9 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt index 397a3eb10c..f20c8c4a0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt @@ -402,7 +402,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -423,7 +423,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt index affbc9ef37..a3e6b17491 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt index 0d18ca5626..d24cb217a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt index 397a3eb10c..f20c8c4a0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt @@ -402,7 +402,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -423,7 +423,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt index affbc9ef37..a3e6b17491 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0d18ca5626..d24cb217a9 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/explain.txt index 587f232ea8..954779ae6b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometSort (34) +- CometColumnarExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) :- * Filter (25) : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -20,17 +20,17 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.customer (26) @@ -44,23 +44,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] @@ -79,20 +79,20 @@ Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND is Input [2]: [s_store_sk#8, s_county#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (11) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] @@ -111,24 +111,24 @@ Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apac Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#10] (18) BroadcastExchange Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] @@ -139,17 +139,17 @@ Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) @@ -168,20 +168,20 @@ Condition : isnotnull(c_customer_sk#18) Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] (30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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] @@ -193,14 +193,14 @@ Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC 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: [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 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] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (40) -+- * CometColumnarToRow (39) ++- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometNativeScan parquet spark_catalog.default.date_dim (36) @@ -221,7 +221,7 @@ Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(39) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (40) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt index 9e4596c12c..40230582bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- BroadcastHashJoin :- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,27 +19,27 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/simplified.txt index 63c6ac3645..bf781c83d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/simplified.txt @@ -1,59 +1,49 @@ -WholeStageCodegen (7) - 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 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] +CometNativeColumnarToRow + 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 + WholeStageCodegen (2) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt index 50af5f4f75..b114cc7dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt index d57afd4fcf..ee10f76856 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 50af5f4f75..b114cc7dd5 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 @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d57afd4fcf..ee10f76856 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/explain.txt index 0b89e61f7b..c4100ac2bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -13,7 +13,7 @@ TakeOrderedAndProject (45) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -35,12 +35,12 @@ TakeOrderedAndProject (45) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_address (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) @@ -57,7 +57,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] (6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] @@ -87,7 +87,7 @@ Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi @@ -100,19 +100,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] (13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] @@ -120,7 +120,7 @@ Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) @@ -133,19 +133,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] @@ -153,17 +153,17 @@ Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(25) Filter [codegen id : 4] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -182,20 +182,20 @@ Condition : isnotnull(ca_address_sk#16) Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) AS ca_state#18] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [2]: [ca_address_sk#16, ca_state#18] (31) BroadcastExchange Input [2]: [ca_address_sk#16, ca_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, ca_state#18] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#18] @@ -214,24 +214,24 @@ Condition : isnotnull(cd_demo_sk#19) Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#20, 1, true, false, true) AS cd_gender#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] (38) BroadcastExchange Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 4] Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Input [8]: [c_current_cdemo_sk#4, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 4] Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#22), partial_max(cd_dep_count#22), partial_sum(cd_dep_count#22), partial_avg(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_sum(cd_dep_employed_count#23), partial_avg(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_sum(cd_dep_college_count#24)] @@ -242,10 +242,10 @@ Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 5] Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [count(1), avg(cd_dep_count#22), max(cd_dep_count#22), sum(cd_dep_count#22), avg(cd_dep_employed_count#23), max(cd_dep_employed_count#23), sum(cd_dep_employed_count#23), avg(cd_dep_college_count#24), max(cd_dep_college_count#24), sum(cd_dep_college_count#24)] @@ -260,7 +260,7 @@ Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * CometColumnarToRow (49) ++- CometNativeColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometNativeScan parquet spark_catalog.default.date_dim (46) @@ -281,7 +281,7 @@ Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 2 Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt index 8935b6cc13..40a8ef4a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -22,12 +22,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,17 +49,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/simplified.txt index 0fb95102c3..e7163949cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -29,17 +29,15 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,7 +48,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -61,17 +59,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt index b43ed9b192..148f71c48d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#20) Input [2]: [ca_address_sk#20, ca_state#21] Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#20, ca_state#22] (33) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#22] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#23) Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt index fea6a32741..af19f84961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt index b43ed9b192..148f71c48d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#20) Input [2]: [ca_address_sk#20, ca_state#21] Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#20, ca_state#22] (33) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#22] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#23) Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt index fea6a32741..af19f84961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/explain.txt index 4ff78399c8..3d214164af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -11,7 +11,7 @@ TakeOrderedAndProject (43) : :- * Project (24) : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) + : : : :- CometNativeColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : +- BroadcastExchange (9) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- Scan parquet spark_catalog.default.catalog_sales (16) : : +- ReusedExchange (18) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer_address (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) @@ -55,7 +55,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] (6) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] @@ -85,7 +85,7 @@ Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi @@ -98,19 +98,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] (13) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#10] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#8 AS customsk#11] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] @@ -121,19 +121,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] +(17) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] (18) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#14] -(19) BroadcastHashJoin [codegen id : 6] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(20) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#12 AS customsk#15] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] @@ -143,13 +143,13 @@ Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] Input [1]: [customsk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#11] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 9] +(24) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -168,20 +168,20 @@ Condition : isnotnull(ca_address_sk#16) Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) AS ca_state#18] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [2]: [ca_address_sk#16, ca_state#18] (29) BroadcastExchange Input [2]: [ca_address_sk#16, ca_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(31) Project [codegen id : 9] +(31) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, ca_state#18] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16, ca_state#18] @@ -200,24 +200,24 @@ Condition : isnotnull(cd_demo_sk#19) Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#20, 1, true, false, true) AS cd_gender#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(35) CometColumnarToRow [codegen id : 8] +(35) CometNativeColumnarToRow Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] (36) BroadcastExchange Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 4] Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Input [8]: [c_current_cdemo_sk#2, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(39) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 4] Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#22), partial_max(cd_dep_count#22), partial_sum(cd_dep_count#22), partial_avg(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_sum(cd_dep_employed_count#23), partial_avg(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_sum(cd_dep_college_count#24)] @@ -228,10 +228,10 @@ Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 5] Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [count(1), avg(cd_dep_count#22), max(cd_dep_count#22), sum(cd_dep_count#22), avg(cd_dep_employed_count#23), max(cd_dep_employed_count#23), sum(cd_dep_employed_count#23), avg(cd_dep_college_count#24), max(cd_dep_college_count#24), sum(cd_dep_college_count#24)] @@ -246,7 +246,7 @@ Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -267,7 +267,7 @@ Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 1 Input [3]: [d_date_sk#7, d_year#75, d_qoy#76] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt index 8aebe6be9a..9091536e96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : :- Project : : +- BroadcastHashJoin : : :- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange @@ -20,12 +20,12 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,7 +37,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -47,17 +47,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/simplified.txt index bed891a0a3..4fe5cb9c56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,13 +13,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -27,18 +27,16 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 Union - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -47,7 +45,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -58,17 +56,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt index e22a5c492c..07c4d37452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt @@ -232,7 +232,7 @@ Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, c Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999 Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt index 69e8d4868c..15668eeef0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt index e22a5c492c..07c4d37452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt @@ -232,7 +232,7 @@ Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, c Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999 Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 69e8d4868c..15668eeef0 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/explain.txt index 7911d900e1..a482e88d1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- CometExchange (43) +- CometHashAggregate (42) @@ -10,7 +10,7 @@ TakeOrderedAndProject (48) +- * HashAggregate (40) +- Union (39) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -24,28 +24,28 @@ TakeOrderedAndProject (48) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.item (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.store (14) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) + : +- CometNativeColumnarToRow (26) : +- ReusedExchange (25) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * HashAggregate (34) - +- * CometColumnarToRow (33) + +- CometNativeColumnarToRow (33) +- ReusedExchange (32) @@ -57,23 +57,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (4) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] @@ -92,20 +92,20 @@ Condition : isnotnull(i_item_sk#8) Input [3]: [i_item_sk#8, i_class#9, i_category#10] Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#8, i_class#11, i_category#12] (11) BroadcastExchange Input [3]: [i_item_sk#8, i_class#11, i_category#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] @@ -124,24 +124,24 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#13, s_state#14] Arguments: [s_store_sk#13], [s_store_sk#13] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [s_store_sk#13] (18) BroadcastExchange Input [1]: [s_store_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Keys [2]: [i_category#12, i_class#11] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -152,10 +152,10 @@ Results [4]: [i_category#12, i_class#11, sum#17, sum#18] Input [4]: [i_category#12, i_class#11, sum#17, sum#18] Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 5] +(23) CometNativeColumnarToRow Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [4]: [i_category#12, i_class#11, sum#17, sum#18] Keys [2]: [i_category#12, i_class#11] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] @@ -165,17 +165,17 @@ Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / M (25) ReusedExchange [Reuses operator id: 22] Output [4]: [i_category#12, i_class#11, sum#27, sum#28] -(26) CometColumnarToRow [codegen id : 10] +(26) CometNativeColumnarToRow Input [4]: [i_category#12, i_class#11, sum#27, sum#28] -(27) HashAggregate [codegen id : 10] +(27) HashAggregate [codegen id : 4] Input [4]: [i_category#12, i_class#11, sum#27, sum#28] Keys [2]: [i_category#12, i_class#11] Functions [2]: [sum(UnscaledValue(ss_net_profit#29)), sum(UnscaledValue(ss_ext_sales_price#30))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#29))#31, sum(UnscaledValue(ss_ext_sales_price#30))#32] Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#29))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#30))#32,17,2) AS ss_ext_sales_price#34, i_category#12] -(28) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 4] Input [3]: [ss_net_profit#33, ss_ext_sales_price#34, i_category#12] Keys [1]: [i_category#12] Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] @@ -186,10 +186,10 @@ Results [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 11] +(30) CometNativeColumnarToRow Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -(31) HashAggregate [codegen id : 11] +(31) HashAggregate [codegen id : 5] Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] Keys [1]: [i_category#12] Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] @@ -199,17 +199,17 @@ Results [6]: [cast((sum(ss_net_profit#33)#43 / sum(ss_ext_sales_price#34)#44) as (32) ReusedExchange [Reuses operator id: 22] Output [4]: [i_category#12, i_class#11, sum#50, sum#51] -(33) CometColumnarToRow [codegen id : 16] +(33) CometNativeColumnarToRow Input [4]: [i_category#12, i_class#11, sum#50, sum#51] -(34) HashAggregate [codegen id : 16] +(34) HashAggregate [codegen id : 7] Input [4]: [i_category#12, i_class#11, sum#50, sum#51] Keys [2]: [i_category#12, i_class#11] Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#31, sum(UnscaledValue(ss_ext_sales_price#53))#32] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#31,17,2) AS ss_net_profit#54, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#32,17,2) AS ss_ext_sales_price#55] -(35) HashAggregate [codegen id : 16] +(35) HashAggregate [codegen id : 7] Input [2]: [ss_net_profit#54, ss_ext_sales_price#55] Keys: [] Functions [2]: [partial_sum(ss_net_profit#54), partial_sum(ss_ext_sales_price#55)] @@ -220,10 +220,10 @@ Results [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 17] +(37) CometNativeColumnarToRow Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -(38) HashAggregate [codegen id : 17] +(38) HashAggregate [codegen id : 8] Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] Keys: [] Functions [2]: [sum(ss_net_profit#54), sum(ss_ext_sales_price#55)] @@ -232,7 +232,7 @@ Results [6]: [cast((sum(ss_net_profit#54)#64 / sum(ss_ext_sales_price#55)#65) as (39) Union -(40) HashAggregate [codegen id : 18] +(40) HashAggregate [codegen id : 9] Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] Functions: [] @@ -256,14 +256,14 @@ Arguments: hashpartitioning(lochierarchy#26, _w0#72, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72], [lochierarchy#26 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 19] +(45) CometNativeColumnarToRow Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] (46) Window Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#72, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#26, _w0#72], [gross_margin#21 ASC NULLS FIRST] -(47) Project [codegen id : 20] +(47) Project [codegen id : 10] Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72, rank_within_parent#73] @@ -275,7 +275,7 @@ Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -296,7 +296,7 @@ Condition : ((isnotnull(d_year#74) AND (d_year#74 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#7, d_year#74] Arguments: [d_date_sk#7], [d_date_sk#7] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt index 7d9e9883d2..1fc86dc5a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -9,7 +9,7 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -23,31 +23,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -61,31 +61,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -99,22 +99,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/simplified.txt index 65728fc6f2..a101d9fb6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/simplified.txt @@ -1,81 +1,73 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (20) + WholeStageCodegen (10) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - 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 (18) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + 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 (9) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (5) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category] #7 + WholeStageCodegen (4) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (8) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #8 + WholeStageCodegen (7) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt index 2562693a01..88c56b6e95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt @@ -244,7 +244,7 @@ Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -265,7 +265,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt index 1f6c984b22..968ea0611f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,7 +53,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -85,7 +85,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt index e91b278c14..9fddc4902a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt @@ -25,12 +25,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt index 2562693a01..88c56b6e95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt @@ -244,7 +244,7 @@ Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -265,7 +265,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt index 1f6c984b22..968ea0611f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,7 +53,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -85,7 +85,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e91b278c14..9fddc4902a 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 @@ -25,12 +25,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/explain.txt index 9613d03c41..67b73fe5c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.store_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.store (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.store_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#6] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] (17) BroadcastExchange Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#7] Right keys [1]: [s_store_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] @@ -151,10 +151,10 @@ Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_yea Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] (27) Window Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] (32) ReusedExchange [Reuses operator id: 21] Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_ Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] (38) Window Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] @@ -228,13 +228,13 @@ Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year# Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] @@ -245,14 +245,14 @@ Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] (46) Window Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] @@ -260,13 +260,13 @@ Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year# Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt index 092083e465..ab44bbef5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/simplified.txt index ee8e4dd40c..404ef436b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk,s_store_name,s_company_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt index 7b1f860f32..bb0182c8d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt index a62c33ecc1..8a10571eff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt index 7b1f860f32..bb0182c8d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 a62c33ecc1..8a10571eff 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/explain.txt index 070113d8ea..9f67fa1915 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (84) +CometNativeColumnarToRow (84) +- CometTakeOrderedAndProject (83) +- CometHashAggregate (82) +- CometColumnarExchange (81) @@ -10,11 +10,11 @@ : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometSort (20) : +- CometColumnarExchange (19) : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometColumnarExchange (16) : +- * HashAggregate (15) : +- * Project (14) @@ -26,7 +26,7 @@ : : : +- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) + : : +- CometNativeColumnarToRow (9) : : +- CometProject (8) : : +- CometFilter (7) : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) @@ -36,11 +36,11 @@ : +- Window (50) : +- * Sort (49) : +- Window (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometSort (46) : +- CometColumnarExchange (45) : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -52,7 +52,7 @@ : : : +- * Filter (29) : : : +- * ColumnarToRow (28) : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) + : : +- CometNativeColumnarToRow (35) : : +- CometProject (34) : : +- CometFilter (33) : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) @@ -62,11 +62,11 @@ +- Window (76) +- * Sort (75) +- Window (74) - +- * CometColumnarToRow (73) + +- CometNativeColumnarToRow (73) +- CometSort (72) +- CometColumnarExchange (71) +- * HashAggregate (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometColumnarExchange (68) +- * HashAggregate (67) +- * Project (66) @@ -78,7 +78,7 @@ : : +- * Filter (55) : : +- * ColumnarToRow (54) : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) + : +- CometNativeColumnarToRow (61) : +- CometProject (60) : +- CometFilter (59) : +- CometNativeScan parquet spark_catalog.default.store_returns (58) @@ -123,33 +123,33 @@ Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AN Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometColumnarToRow +(9) CometNativeColumnarToRow Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [2]: [ws_order_number#2, ws_item_sk#1] Right keys [2]: [wr_order_number#9, wr_item_sk#8] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] (12) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#13] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#6] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 2] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -160,10 +160,10 @@ Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty# Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -178,14 +178,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 5] +(21) CometNativeColumnarToRow Input [3]: [item#30, return_ratio#31, currency_ratio#32] (22) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 4] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 @@ -193,11 +193,11 @@ Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 5] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(26) Project [codegen id : 7] +(26) Project [codegen id : 5] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -209,14 +209,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -(29) Filter [codegen id : 8] +(29) Filter [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(30) Project [codegen id : 8] +(30) Project [codegen id : 6] Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] @@ -239,33 +239,33 @@ Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.0 Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(35) CometColumnarToRow +(35) CometNativeColumnarToRow Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 7] Left keys [2]: [cs_order_number#37, cs_item_sk#36] Right keys [2]: [cr_order_number#43, cr_item_sk#42] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 7] Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] (38) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#47] -(39) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#47] Join type: Inner Join condition: None -(40) Project [codegen id : 10] +(40) Project [codegen id : 7] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] -(41) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -276,10 +276,10 @@ Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometColumnarToRow [codegen id : 11] +(43) CometNativeColumnarToRow Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 8] Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -294,14 +294,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 12] +(47) CometNativeColumnarToRow Input [3]: [item#64, return_ratio#65, currency_ratio#66] (48) Window Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] -(49) Sort [codegen id : 13] +(49) Sort [codegen id : 9] Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 @@ -309,11 +309,11 @@ Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] -(51) Filter [codegen id : 14] +(51) Filter [codegen id : 10] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) -(52) Project [codegen id : 14] +(52) Project [codegen id : 10] Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] @@ -325,14 +325,14 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 15] +(54) ColumnarToRow [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -(55) Filter [codegen id : 15] +(55) Filter [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) -(56) Project [codegen id : 15] +(56) Project [codegen id : 11] Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] @@ -355,33 +355,33 @@ Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AN Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(61) CometColumnarToRow +(61) CometNativeColumnarToRow Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(62) BroadcastHashJoin [codegen id : 17] +(62) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(63) Project [codegen id : 12] Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] (64) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#81] -(65) BroadcastHashJoin [codegen id : 17] +(65) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ss_sold_date_sk#75] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(66) Project [codegen id : 12] Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] -(67) HashAggregate [codegen id : 17] +(67) HashAggregate [codegen id : 12] Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Keys [1]: [ss_item_sk#70] Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -392,10 +392,10 @@ Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(69) CometColumnarToRow [codegen id : 18] +(69) CometNativeColumnarToRow Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -(70) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 13] Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Keys [1]: [ss_item_sk#70] Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -410,14 +410,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [3]: [item#98, return_ratio#99, currency_ratio#100] (74) Window Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] -(75) Sort [codegen id : 20] +(75) Sort [codegen id : 14] Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 @@ -425,17 +425,17 @@ Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] -(77) Filter [codegen id : 21] +(77) Filter [codegen id : 15] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) -(78) Project [codegen id : 21] +(78) Project [codegen id : 15] Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] (79) Union -(80) HashAggregate [codegen id : 22] +(80) HashAggregate [codegen id : 16] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] @@ -455,14 +455,14 @@ Functions: [] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) CometColumnarToRow [codegen id : 23] +(84) CometNativeColumnarToRow Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometNativeScan parquet spark_catalog.default.date_dim (85) @@ -483,7 +483,7 @@ Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 Input [3]: [d_date_sk#13, d_year#104, d_moy#105] Arguments: [d_date_sk#13], [d_date_sk#13] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt index 6c4ded1515..0090739d55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -9,11 +9,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -27,16 +27,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -45,11 +45,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -62,12 +62,12 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -76,11 +76,11 @@ CometColumnarToRow +- Window +- Sort +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -93,12 +93,12 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/simplified.txt index 047269434d..018aec1d47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/simplified.txt @@ -1,140 +1,130 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + CometNativeColumnarToRow + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #6 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [cs_item_sk] #7 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (15) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #9 + WholeStageCodegen (13) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (12) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (11) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt index 74702d596d..b027d714df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt index ca80833ee5..3353e1c331 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt index 74702d596d..b027d714df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ca80833ee5..3353e1c331 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 @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/explain.txt index 23adfe692b..178350b771 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (77) +- * Project (73) +- * BroadcastHashJoin Inner BuildRight (72) :- Window (65) - : +- * CometColumnarToRow (64) + : +- CometNativeColumnarToRow (64) : +- CometSort (63) : +- CometExchange (62) : +- CometProject (61) @@ -15,18 +15,18 @@ TakeOrderedAndProject (77) : :- CometSort (29) : : +- CometColumnarExchange (28) : : +- * HashAggregate (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometColumnarExchange (25) : : +- * HashAggregate (24) : : +- * Project (23) : : +- * BroadcastHashJoin Inner BuildRight (22) : : :- * Project (15) : : : +- Window (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometSort (12) : : : +- CometColumnarExchange (11) : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometColumnarExchange (8) : : : +- * HashAggregate (7) : : : +- * Project (6) @@ -38,24 +38,24 @@ TakeOrderedAndProject (77) : : +- BroadcastExchange (21) : : +- * Project (20) : : +- Window (19) - : : +- * CometColumnarToRow (18) + : : +- CometNativeColumnarToRow (18) : : +- CometSort (17) : : +- ReusedExchange (16) : +- CometSort (58) : +- CometColumnarExchange (57) : +- * HashAggregate (56) - : +- * CometColumnarToRow (55) + : +- CometNativeColumnarToRow (55) : +- CometColumnarExchange (54) : +- * HashAggregate (53) : +- * Project (52) : +- * BroadcastHashJoin Inner BuildRight (51) : :- * Project (44) : : +- Window (43) - : : +- * CometColumnarToRow (42) + : : +- CometNativeColumnarToRow (42) : : +- CometSort (41) : : +- CometColumnarExchange (40) : : +- * HashAggregate (39) - : : +- * CometColumnarToRow (38) + : : +- CometNativeColumnarToRow (38) : : +- CometColumnarExchange (37) : : +- * HashAggregate (36) : : +- * Project (35) @@ -67,13 +67,13 @@ TakeOrderedAndProject (77) : +- BroadcastExchange (50) : +- * Project (49) : +- Window (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometSort (46) : +- ReusedExchange (45) +- BroadcastExchange (71) +- * Project (70) +- Window (69) - +- * CometColumnarToRow (68) + +- CometNativeColumnarToRow (68) +- CometSort (67) +- ReusedExchange (66) @@ -86,27 +86,27 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 82] Output [2]: [d_date_sk#5, d_date#6] -(5) BroadcastHashJoin [codegen id : 2] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 2] +(6) Project [codegen id : 1] Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -(7) HashAggregate [codegen id : 2] +(7) HashAggregate [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] @@ -117,10 +117,10 @@ Results [3]: [ws_item_sk#1, d_date#6, sum#8] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 3] +(9) CometNativeColumnarToRow Input [3]: [ws_item_sk#1, d_date#6, sum#8] -(10) HashAggregate [codegen id : 3] +(10) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] @@ -135,14 +135,14 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 4] +(13) CometNativeColumnarToRow Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] (14) Window Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 10] +(15) Project [codegen id : 6] Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] @@ -153,14 +153,14 @@ Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] -(18) CometColumnarToRow [codegen id : 8] +(18) CometNativeColumnarToRow Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] (19) Window Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] -(20) Project [codegen id : 9] +(20) Project [codegen id : 5] Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] @@ -168,17 +168,17 @@ Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] Input [3]: [item_sk#16, sumws#17, rk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 10] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_sk#10] Right keys [1]: [item_sk#16] Join type: Inner Join condition: (rk#12 >= rk#15) -(23) Project [codegen id : 10] +(23) Project [codegen id : 6] Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] -(24) HashAggregate [codegen id : 10] +(24) HashAggregate [codegen id : 6] Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [partial_sum(sumws#17)] @@ -189,10 +189,10 @@ Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometColumnarToRow [codegen id : 11] +(26) CometNativeColumnarToRow Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(27) HashAggregate [codegen id : 11] +(27) HashAggregate [codegen id : 7] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [sum(sumws#17)] @@ -215,27 +215,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 13] +(31) ColumnarToRow [codegen id : 8] Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -(32) Filter [codegen id : 13] +(32) Filter [codegen id : 8] Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#24) (33) ReusedExchange [Reuses operator id: 82] Output [2]: [d_date_sk#27, d_date#28] -(34) BroadcastHashJoin [codegen id : 13] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(35) Project [codegen id : 13] +(35) Project [codegen id : 8] Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27, d_date#28] -(36) HashAggregate [codegen id : 13] +(36) HashAggregate [codegen id : 8] Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] Keys [2]: [ss_item_sk#24, d_date#28] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] @@ -246,10 +246,10 @@ Results [3]: [ss_item_sk#24, d_date#28, sum#30] Input [3]: [ss_item_sk#24, d_date#28, sum#30] Arguments: hashpartitioning(ss_item_sk#24, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(38) CometColumnarToRow [codegen id : 14] +(38) CometNativeColumnarToRow Input [3]: [ss_item_sk#24, d_date#28, sum#30] -(39) HashAggregate [codegen id : 14] +(39) HashAggregate [codegen id : 9] Input [3]: [ss_item_sk#24, d_date#28, sum#30] Keys [2]: [ss_item_sk#24, d_date#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] @@ -264,14 +264,14 @@ Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumna Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] Arguments: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(42) CometColumnarToRow [codegen id : 15] +(42) CometNativeColumnarToRow Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] (43) Window Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#28 ASC NULLS FIRST] -(44) Project [codegen id : 21] +(44) Project [codegen id : 13] Output [4]: [item_sk#32, d_date#28, sumss#33, rk#34] Input [5]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24, rk#34] @@ -282,14 +282,14 @@ Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] Arguments: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36], [ss_item_sk#36 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 19] +(47) CometNativeColumnarToRow Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] (48) Window Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#35 ASC NULLS FIRST] -(49) Project [codegen id : 20] +(49) Project [codegen id : 12] Output [3]: [item_sk#32 AS item_sk#38, sumss#33 AS sumss#39, rk#37] Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] @@ -297,17 +297,17 @@ Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] Input [3]: [item_sk#38, sumss#39, rk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 21] +(51) BroadcastHashJoin [codegen id : 13] Left keys [1]: [item_sk#32] Right keys [1]: [item_sk#38] Join type: Inner Join condition: (rk#34 >= rk#37) -(52) Project [codegen id : 21] +(52) Project [codegen id : 13] Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#38, sumss#39, rk#37] -(53) HashAggregate [codegen id : 21] +(53) HashAggregate [codegen id : 13] Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] Keys [3]: [item_sk#32, d_date#28, sumss#33] Functions [1]: [partial_sum(sumss#39)] @@ -318,10 +318,10 @@ Results [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Arguments: hashpartitioning(item_sk#32, d_date#28, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(55) CometColumnarToRow [codegen id : 22] +(55) CometNativeColumnarToRow Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] -(56) HashAggregate [codegen id : 22] +(56) HashAggregate [codegen id : 14] Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Keys [3]: [item_sk#32, d_date#28, sumss#33] Functions [1]: [sum(sumss#39)] @@ -357,7 +357,7 @@ Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] -(64) CometColumnarToRow [codegen id : 23] +(64) CometNativeColumnarToRow Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] (65) Window @@ -371,14 +371,14 @@ Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] -(68) CometColumnarToRow [codegen id : 46] +(68) CometNativeColumnarToRow Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] (69) Window Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#46], [d_date#47 ASC NULLS FIRST] -(70) Project [codegen id : 47] +(70) Project [codegen id : 29] Output [4]: [item_sk#46 AS item_sk#52, web_sales#48 AS web_sales#53, store_sales#49 AS store_sales#54, rk#51] Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] @@ -386,31 +386,31 @@ Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] Input [4]: [item_sk#52, web_sales#53, store_sales#54, rk#51] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(72) BroadcastHashJoin [codegen id : 48] +(72) BroadcastHashJoin [codegen id : 30] Left keys [1]: [item_sk#46] Right keys [1]: [item_sk#52] Join type: Inner Join condition: (rk#50 >= rk#51) -(73) Project [codegen id : 48] +(73) Project [codegen id : 30] Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#52, web_sales#53, store_sales#54, rk#51] -(74) HashAggregate [codegen id : 48] +(74) HashAggregate [codegen id : 30] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] Aggregate Attributes [2]: [max#55, max#56] Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] -(75) HashAggregate [codegen id : 48] +(75) HashAggregate [codegen id : 30] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [max(web_sales#53), max(store_sales#54)] Aggregate Attributes [2]: [max(web_sales#53)#59, max(store_sales#54)#60] Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#59 AS web_cumulative#61, max(store_sales#54)#60 AS store_cumulative#62] -(76) Filter [codegen id : 48] +(76) Filter [codegen id : 30] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] Condition : ((isnotnull(web_cumulative#61) AND isnotnull(store_cumulative#62)) AND (web_cumulative#61 > store_cumulative#62)) @@ -422,7 +422,7 @@ Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (82) -+- * CometColumnarToRow (81) ++- CometNativeColumnarToRow (81) +- CometProject (80) +- CometFilter (79) +- CometNativeScan parquet spark_catalog.default.date_dim (78) @@ -443,7 +443,7 @@ Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_mo Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(81) CometColumnarToRow [codegen id : 1] +(81) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (82) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt index 8eb3f067e2..92ee33eda6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject +- Project +- BroadcastHashJoin :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometExchange : +- CometProject @@ -14,18 +14,18 @@ TakeOrderedAndProject : :- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometSort : : : +- CometColumnarExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -35,23 +35,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -61,30 +61,30 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -94,18 +94,18 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -115,14 +115,14 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -131,18 +131,18 @@ TakeOrderedAndProject :- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -152,23 +152,23 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -178,30 +178,30 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -211,18 +211,18 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -232,7 +232,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/simplified.txt index 00cd0a509c..55c9b21954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (48) + WholeStageCodegen (30) Filter [web_cumulative,store_cumulative] HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] @@ -7,121 +7,107 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #8 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (21) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #11 - WholeStageCodegen (13) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (20) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + CometNativeColumnarToRow + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (7) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (6) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk,d_date] #5 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #8 + WholeStageCodegen (14) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [item_sk,d_date,sumss] #9 + WholeStageCodegen (13) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (9) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk,d_date] #11 + WholeStageCodegen (8) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (12) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter BroadcastExchange #13 - WholeStageCodegen (47) + WholeStageCodegen (29) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (46) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometNativeColumnarToRow + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt index f8a95a6db8..a894d9e3e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometProject (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -436,7 +436,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt index f7d3371108..05c8752bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt @@ -33,7 +33,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -138,7 +138,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -161,7 +161,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt index b3013059b0..bcb2317588 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt @@ -43,12 +43,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt index f8a95a6db8..a894d9e3e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt @@ -415,7 +415,7 @@ Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometProject (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -436,7 +436,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt index f7d3371108..05c8752bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt @@ -33,7 +33,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -138,7 +138,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -161,7 +161,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 b3013059b0..bcb2317588 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 @@ -43,12 +43,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/explain.txt index be90722d2e..e3e412b4ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.call_center (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.catalog_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [2]: [cc_call_center_sk#14, cc_name#15] (17) BroadcastExchange Input [2]: [cc_call_center_sk#14, cc_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_call_center_sk#6] Right keys [1]: [cc_call_center_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] @@ -151,10 +151,10 @@ Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] (27) Window Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] (32) ReusedExchange [Reuses operator id: 21] Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQU Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] (38) Window Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] @@ -228,13 +228,13 @@ Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales# Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] @@ -245,14 +245,14 @@ Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] (46) Window Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] @@ -260,13 +260,13 @@ Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales# Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt index 65c66a7da8..66c510aaa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/simplified.txt index e603f05322..b61b5372fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [cs_item_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [cc_call_center_sk,cc_name] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt index d1daee98c1..fcd3657557 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt index d655789fe6..d89e7759f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt index d1daee98c1..fcd3657557 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 d655789fe6..d89e7759f3 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/explain.txt index 62a1484a64..1e763cfdc4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/explain.txt @@ -1,17 +1,17 @@ == Physical Plan == -* CometColumnarToRow (99) +CometNativeColumnarToRow (99) +- CometTakeOrderedAndProject (98) +- CometHashAggregate (97) +- CometColumnarExchange (96) +- * HashAggregate (95) +- Union (94) :- * HashAggregate (79) - : +- * CometColumnarToRow (78) + : +- CometNativeColumnarToRow (78) : +- CometColumnarExchange (77) : +- * HashAggregate (76) : +- Union (75) : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -29,12 +29,12 @@ : : : : +- Scan parquet spark_catalog.default.store_returns (5) : : : +- ReusedExchange (10) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometProject (15) : : +- CometFilter (14) : : +- CometNativeScan parquet spark_catalog.default.store (13) : :- * HashAggregate (46) - : : +- * CometColumnarToRow (45) + : : +- CometNativeColumnarToRow (45) : : +- CometColumnarExchange (44) : : +- * HashAggregate (43) : : +- * Project (42) @@ -52,12 +52,12 @@ : : : : +- Scan parquet spark_catalog.default.catalog_returns (28) : : : +- ReusedExchange (33) : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) + : : +- CometNativeColumnarToRow (39) : : +- CometProject (38) : : +- CometFilter (37) : : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) : +- * HashAggregate (74) - : +- * CometColumnarToRow (73) + : +- CometNativeColumnarToRow (73) : +- CometColumnarExchange (72) : +- * HashAggregate (71) : +- * Project (70) @@ -74,29 +74,29 @@ : : : :- BroadcastExchange (53) : : : : +- * ColumnarToRow (52) : : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : : +- * CometColumnarToRow (57) + : : : +- CometNativeColumnarToRow (57) : : : +- CometProject (56) : : : +- CometFilter (55) : : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) : : +- ReusedExchange (61) : +- BroadcastExchange (68) - : +- * CometColumnarToRow (67) + : +- CometNativeColumnarToRow (67) : +- CometProject (66) : +- CometFilter (65) : +- CometNativeScan parquet spark_catalog.default.web_site (64) :- * HashAggregate (86) - : +- * CometColumnarToRow (85) + : +- CometNativeColumnarToRow (85) : +- CometColumnarExchange (84) : +- * HashAggregate (83) : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) + : +- CometNativeColumnarToRow (81) : +- ReusedExchange (80) +- * HashAggregate (93) - +- * CometColumnarToRow (92) + +- CometNativeColumnarToRow (92) +- CometColumnarExchange (91) +- * HashAggregate (90) +- * HashAggregate (89) - +- * CometColumnarToRow (88) + +- CometNativeColumnarToRow (88) +- ReusedExchange (87) @@ -143,13 +143,13 @@ Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_s (10) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#22] -(11) BroadcastHashJoin [codegen id : 5] +(11) BroadcastHashJoin [codegen id : 3] Left keys [1]: [date_sk#7] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(12) Project [codegen id : 5] +(12) Project [codegen id : 3] Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] @@ -168,24 +168,24 @@ Condition : isnotnull(s_store_sk#23) Input [2]: [s_store_sk#23, s_store_id#24] Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#23, s_store_id#25] (17) BroadcastExchange Input [2]: [s_store_sk#23, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 5] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [store_sk#6] Right keys [1]: [s_store_sk#23] Join type: Inner Join condition: None -(19) Project [codegen id : 5] +(19) Project [codegen id : 3] Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] -(20) HashAggregate [codegen id : 5] +(20) HashAggregate [codegen id : 3] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] @@ -196,10 +196,10 @@ Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(23) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 4] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -214,14 +214,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] +(25) ColumnarToRow [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -(26) Filter [codegen id : 7] +(26) Filter [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Condition : isnotnull(cs_catalog_page_sk#43) -(27) Project [codegen id : 7] +(27) Project [codegen id : 5] Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] @@ -233,14 +233,14 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 8] +(29) ColumnarToRow [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -(30) Filter [codegen id : 8] +(30) Filter [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(31) Project [codegen id : 8] +(31) Project [codegen id : 6] Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] @@ -249,13 +249,13 @@ Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_retur (33) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#63] -(34) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(35) Project [codegen id : 11] +(35) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -274,24 +274,24 @@ Condition : isnotnull(cp_catalog_page_sk#64) Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#65, 16, true, false, true) AS cp_catalog_page_id#66] -(39) CometColumnarToRow [codegen id : 10] +(39) CometNativeColumnarToRow Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] (40) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(41) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(42) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] @@ -302,10 +302,10 @@ Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(45) CometColumnarToRow [codegen id : 12] +(45) CometNativeColumnarToRow Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -(46) HashAggregate [codegen id : 12] +(46) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -320,14 +320,14 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 13] +(48) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(49) Filter [codegen id : 13] +(49) Filter [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Condition : isnotnull(ws_web_site_sk#84) -(50) Project [codegen id : 13] +(50) Project [codegen id : 9] Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] @@ -338,7 +338,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 14] +(52) ColumnarToRow [codegen id : 10] Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (53) BroadcastExchange @@ -360,16 +360,16 @@ Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND i Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(57) CometColumnarToRow +(57) CometNativeColumnarToRow Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(58) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 11] Left keys [2]: [wr_item_sk#94, wr_order_number#95] Right keys [2]: [ws_item_sk#99, ws_order_number#101] Join type: Inner Join condition: None -(59) Project [codegen id : 15] +(59) Project [codegen id : 11] Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] @@ -378,13 +378,13 @@ Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, (61) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#109] -(62) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 12] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(63) Project [codegen id : 18] +(63) Project [codegen id : 12] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -403,24 +403,24 @@ Condition : isnotnull(web_site_sk#110) Input [2]: [web_site_sk#110, web_site_id#111] Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#111, 16, true, false, true) AS web_site_id#112] -(67) CometColumnarToRow [codegen id : 17] +(67) CometNativeColumnarToRow Input [2]: [web_site_sk#110, web_site_id#112] (68) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(69) BroadcastHashJoin [codegen id : 18] +(69) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(70) Project [codegen id : 18] +(70) Project [codegen id : 12] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] -(71) HashAggregate [codegen id : 18] +(71) HashAggregate [codegen id : 12] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Keys [1]: [web_site_id#112] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] @@ -431,10 +431,10 @@ Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -(74) HashAggregate [codegen id : 19] +(74) HashAggregate [codegen id : 13] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Keys [1]: [web_site_id#112] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] @@ -443,7 +443,7 @@ Results [5]: [web channel AS channel#125, concat(web_site, web_site_id#112) AS i (75) Union -(76) HashAggregate [codegen id : 20] +(76) HashAggregate [codegen id : 14] Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] Keys [2]: [channel#38, id#39] Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] @@ -454,10 +454,10 @@ Results [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(78) CometColumnarToRow [codegen id : 21] +(78) CometNativeColumnarToRow Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(79) HashAggregate [codegen id : 21] +(79) HashAggregate [codegen id : 15] Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Keys [2]: [channel#38, id#39] Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] @@ -467,17 +467,17 @@ Results [5]: [channel#38, id#39, cast(sum(sales#40)#142 as decimal(37,2)) AS sal (80) ReusedExchange [Reuses operator id: 77] Output [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(81) CometColumnarToRow [codegen id : 42] +(81) CometNativeColumnarToRow Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(82) HashAggregate [codegen id : 42] +(82) HashAggregate [codegen id : 30] Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Keys [2]: [channel#38, id#39] Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] Results [4]: [channel#38, sum(sales#40)#142 AS sales#148, sum(returns#41)#143 AS returns#149, sum(profit#42)#144 AS profit#150] -(83) HashAggregate [codegen id : 42] +(83) HashAggregate [codegen id : 30] Input [4]: [channel#38, sales#148, returns#149, profit#150] Keys [1]: [channel#38] Functions [3]: [partial_sum(sales#148), partial_sum(returns#149), partial_sum(profit#150)] @@ -488,10 +488,10 @@ Results [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, i Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Arguments: hashpartitioning(channel#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(85) CometColumnarToRow [codegen id : 43] +(85) CometNativeColumnarToRow Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -(86) HashAggregate [codegen id : 43] +(86) HashAggregate [codegen id : 31] Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Keys [1]: [channel#38] Functions [3]: [sum(sales#148), sum(returns#149), sum(profit#150)] @@ -501,17 +501,17 @@ Results [5]: [channel#38, null AS id#166, sum(sales#148)#163 AS sum(sales)#167, (87) ReusedExchange [Reuses operator id: 77] Output [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(88) CometColumnarToRow [codegen id : 64] +(88) CometNativeColumnarToRow Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(89) HashAggregate [codegen id : 64] +(89) HashAggregate [codegen id : 46] Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Keys [2]: [channel#38, id#39] Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] Results [3]: [sum(sales#40)#142 AS sales#170, sum(returns#41)#143 AS returns#171, sum(profit#42)#144 AS profit#172] -(90) HashAggregate [codegen id : 64] +(90) HashAggregate [codegen id : 46] Input [3]: [sales#170, returns#171, profit#172] Keys: [] Functions [3]: [partial_sum(sales#170), partial_sum(returns#171), partial_sum(profit#172)] @@ -522,10 +522,10 @@ Results [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] Input [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(92) CometColumnarToRow [codegen id : 65] +(92) CometNativeColumnarToRow Input [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] -(93) HashAggregate [codegen id : 65] +(93) HashAggregate [codegen id : 47] Input [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] Keys: [] Functions [3]: [sum(sales#170), sum(returns#171), sum(profit#172)] @@ -534,7 +534,7 @@ Results [5]: [null AS channel#188, null AS id#189, sum(sales#170)#185 AS sum(sal (94) Union -(95) HashAggregate [codegen id : 66] +(95) HashAggregate [codegen id : 48] Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Functions: [] @@ -554,14 +554,14 @@ Functions: [] Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#145,returns#146,profit#147]), [channel#38, id#39, sales#145, returns#146, profit#147], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] -(99) CometColumnarToRow [codegen id : 67] +(99) CometNativeColumnarToRow Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (104) -+- * CometColumnarToRow (103) ++- CometNativeColumnarToRow (103) +- CometProject (102) +- CometFilter (101) +- CometNativeScan parquet spark_catalog.default.date_dim (100) @@ -582,7 +582,7 @@ Condition : (((isnotnull(d_date#193) AND (d_date#193 >= 1998-08-04)) AND (d_date Input [2]: [d_date_sk#22, d_date#193] Arguments: [d_date_sk#22], [d_date_sk#22] -(103) CometColumnarToRow [codegen id : 1] +(103) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (104) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt index 63ea8e5ea8..ec2eef025a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt @@ -1,16 +1,16 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,17 +34,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -63,17 +63,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -92,31 +92,31 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_site :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -130,7 +130,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -140,17 +140,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -169,17 +169,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -198,31 +198,31 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_site +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -236,7 +236,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -246,17 +246,17 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -275,17 +275,17 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.catalog_page +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -304,17 +304,17 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/simplified.txt index dc9c9b1ce7..ae0afc7f07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/simplified.txt @@ -1,167 +1,157 @@ -WholeStageCodegen (67) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (66) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (21) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (20) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (48) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (15) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (14) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_store_id] #3 + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #8 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 + Union + WholeStageCodegen (1) + Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] + Filter [ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + WholeStageCodegen (2) + Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] + Filter [sr_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (43) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #11 - WholeStageCodegen (42) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (65) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (64) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow + CometNativeColumnarToRow + CometColumnarExchange [cp_catalog_page_id] #6 + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (5) + Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] + Filter [cs_catalog_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + WholeStageCodegen (6) + Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] + Filter [cr_catalog_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (13) + HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometNativeColumnarToRow + CometColumnarExchange [web_site_id] #8 + WholeStageCodegen (12) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (9) + Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] + Filter [ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + WholeStageCodegen (11) + Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #10 + CometNativeColumnarToRow + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (31) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel] #11 + WholeStageCodegen (30) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (47) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #12 + WholeStageCodegen (46) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt index 33c3a5de2e..fff4911764 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt @@ -483,7 +483,7 @@ Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) @@ -504,7 +504,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt index 35b86dbff5..24167a2372 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,7 +197,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt index c0b236a57a..0084fda697 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt @@ -23,12 +23,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt index 33c3a5de2e..fff4911764 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt @@ -483,7 +483,7 @@ Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) @@ -504,7 +504,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt index 35b86dbff5..24167a2372 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,7 +197,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 c0b236a57a..0084fda697 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 @@ -23,12 +23,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/explain.txt index a57e1f5e30..514f982c44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (40) +- * Filter (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * Project (34) @@ -11,7 +11,7 @@ TakeOrderedAndProject (40) : +- * BroadcastHashJoin Inner BuildRight (17) : :- * Project (15) : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) + : : :- CometNativeColumnarToRow (9) : : : +- CometProject (8) : : : +- CometBroadcastHashJoin (7) : : : :- CometProject (3) @@ -26,7 +26,7 @@ TakeOrderedAndProject (40) : : +- Scan parquet spark_catalog.default.store_sales (10) : +- ReusedExchange (16) +- BroadcastExchange (32) - +- * CometColumnarToRow (31) + +- CometNativeColumnarToRow (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (20) @@ -80,7 +80,7 @@ Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(9) CometColumnarToRow [codegen id : 4] +(9) CometNativeColumnarToRow Input [2]: [ca_state#3, c_customer_sk#4] (10) Scan parquet spark_catalog.default.store_sales @@ -102,26 +102,26 @@ Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#4] Right keys [1]: [ss_customer_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] (16) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 2] Output [2]: [ca_state#3, ss_item_sk#6] Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] @@ -181,24 +181,24 @@ Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCod Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] Arguments: [i_item_sk#11], [i_item_sk#11] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [i_item_sk#11] (32) BroadcastExchange Input [1]: [i_item_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 4] +(33) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#6] Right keys [1]: [i_item_sk#11] Join type: Inner Join condition: None -(34) Project [codegen id : 4] +(34) Project [codegen id : 2] Output [1]: [ca_state#3] Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] -(35) HashAggregate [codegen id : 4] +(35) HashAggregate [codegen id : 2] Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] @@ -209,17 +209,17 @@ Results [2]: [ca_state#3, count#21] Input [2]: [ca_state#3, count#21] Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometColumnarToRow [codegen id : 5] +(37) CometNativeColumnarToRow Input [2]: [ca_state#3, count#21] -(38) HashAggregate [codegen id : 5] +(38) HashAggregate [codegen id : 3] Input [2]: [ca_state#3, count#21] Keys [1]: [ca_state#3] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] Results [3]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24, ca_state#3] -(39) Filter [codegen id : 5] +(39) Filter [codegen id : 3] Input [3]: [state#23, cnt#24, ca_state#3] Condition : (cnt#24 >= 10) @@ -231,7 +231,7 @@ Arguments: 100, [cnt#24 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#23, Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#25] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#28] Keys [1]: [d_month_seq#28] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt index fe3b28849a..f2cdb50e03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometProject @@ -25,11 +25,11 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -38,11 +38,11 @@ TakeOrderedAndProject : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -51,7 +51,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/simplified.txt index 76dd43f9c7..9114fcaf65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (5) + WholeStageCodegen (3) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -13,8 +13,8 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ca_state,ss_item_sk,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ca_state,c_customer_sk] CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] CometProject [ca_state] [ca_address_sk,ca_state] @@ -32,38 +32,32 @@ TakeOrderedAndProject [cnt,ca_state,state] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #7 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #8 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_current_price,i_category] + CometFilter [i_current_price,i_category] + CometNativeScan 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/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt index 73dd1d4955..3bfad35eca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#24, cnt#25] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#26] Keys [1]: [d_month_seq#26] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#26] Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt index f5b69fc6cf..20115b61c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt @@ -23,11 +23,11 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt index 2978e30c1c..fd51229f24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt @@ -25,22 +25,18 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt index 73dd1d4955..3bfad35eca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#24, cnt#25] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#26] Keys [1]: [d_month_seq#26] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#26] Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt index f5b69fc6cf..20115b61c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt @@ -23,11 +23,11 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate 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 2978e30c1c..fd51229f24 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 @@ -25,22 +25,18 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/explain.txt index 135549db3d..322821e984 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (180) +CometNativeColumnarToRow (180) +- CometSort (179) +- CometExchange (178) +- CometProject (177) @@ -48,7 +48,7 @@ : : : : : : : : : : : : : : : : : : +- * Filter (3) : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (8) : : : : : : : : : : : : : : : : : +- CometProject (7) : : : : : : : : : : : : : : : : : +- CometFilter (6) : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) @@ -153,7 +153,7 @@ : : : : : : : : : : : : : : : : : +- * Filter (108) : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (113) : : : : : : : : : : : : : : : : +- CometProject (112) : : : : : : : : : : : : : : : : +- CometFilter (111) : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) @@ -215,7 +215,7 @@ Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) CometColumnarToRow +(8) CometNativeColumnarToRow Input [2]: [sr_item_sk#14, sr_ticket_number#15] (9) BroadcastHashJoin [codegen id : 2] @@ -694,7 +694,7 @@ Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] -(113) CometColumnarToRow +(113) CometNativeColumnarToRow Input [2]: [sr_item_sk#111, sr_ticket_number#112] (114) BroadcastHashJoin [codegen id : 4] @@ -971,14 +971,14 @@ Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC 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: [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 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] -(180) CometColumnarToRow [codegen id : 5] +(180) CometNativeColumnarToRow 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometNativeScan parquet spark_catalog.default.date_dim (181) @@ -994,7 +994,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (184) BroadcastExchange @@ -1003,7 +1003,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (188) -+- * CometColumnarToRow (187) ++- CometNativeColumnarToRow (187) +- CometFilter (186) +- CometNativeScan parquet spark_catalog.default.date_dim (185) @@ -1019,7 +1019,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(187) CometColumnarToRow [codegen id : 1] +(187) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (188) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt index 6e3f37148b..98db64e5b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -49,10 +49,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -171,10 +171,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/simplified.txt index d933995ffc..c902a59f78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/simplified.txt @@ -1,206 +1,200 @@ -WholeStageCodegen (5) - 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] - 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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_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] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +CometNativeColumnarToRow + 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] + 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + WholeStageCodegen (2) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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_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] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometNativeScan 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + WholeStageCodegen (4) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #21 + WholeStageCodegen (3) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [d_date_sk,d_year] #12 ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt index 077daeca3a..fe232d03f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt index 46fe063430..f1bd37b320 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] 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 077daeca3a..fe232d03f0 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 @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject 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 46fe063430..f1bd37b320 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 @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/explain.txt index 1436e02064..a8a53d8115 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/explain.txt @@ -3,14 +3,14 @@ TakeOrderedAndProject (90) +- * Filter (89) +- Window (88) +- WindowGroupLimit (87) - +- * CometColumnarToRow (86) + +- CometNativeColumnarToRow (86) +- CometSort (85) +- CometColumnarExchange (84) +- WindowGroupLimit (83) +- * Sort (82) +- Union (81) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -24,70 +24,70 @@ TakeOrderedAndProject (90) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.item (14) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) + : +- CometNativeColumnarToRow (26) : +- ReusedExchange (25) :- * HashAggregate (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometColumnarExchange (36) : +- * HashAggregate (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) :- * HashAggregate (45) - : +- * CometColumnarToRow (44) + : +- CometNativeColumnarToRow (44) : +- CometColumnarExchange (43) : +- * HashAggregate (42) : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) + : +- CometNativeColumnarToRow (40) : +- ReusedExchange (39) :- * HashAggregate (52) - : +- * CometColumnarToRow (51) + : +- CometNativeColumnarToRow (51) : +- CometColumnarExchange (50) : +- * HashAggregate (49) : +- * HashAggregate (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- ReusedExchange (46) :- * HashAggregate (59) - : +- * CometColumnarToRow (58) + : +- CometNativeColumnarToRow (58) : +- CometColumnarExchange (57) : +- * HashAggregate (56) : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) + : +- CometNativeColumnarToRow (54) : +- ReusedExchange (53) :- * HashAggregate (66) - : +- * CometColumnarToRow (65) + : +- CometNativeColumnarToRow (65) : +- CometColumnarExchange (64) : +- * HashAggregate (63) : +- * HashAggregate (62) - : +- * CometColumnarToRow (61) + : +- CometNativeColumnarToRow (61) : +- ReusedExchange (60) :- * HashAggregate (73) - : +- * CometColumnarToRow (72) + : +- CometNativeColumnarToRow (72) : +- CometColumnarExchange (71) : +- * HashAggregate (70) : +- * HashAggregate (69) - : +- * CometColumnarToRow (68) + : +- CometNativeColumnarToRow (68) : +- ReusedExchange (67) +- * HashAggregate (80) - +- * CometColumnarToRow (79) + +- CometNativeColumnarToRow (79) +- CometColumnarExchange (78) +- * HashAggregate (77) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- ReusedExchange (74) @@ -99,23 +99,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 95] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] @@ -134,20 +134,20 @@ Condition : isnotnull(s_store_sk#11) Input [2]: [s_store_sk#11, s_store_id#12] Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [s_store_sk#11, s_store_id#13] (11) BroadcastExchange Input [2]: [s_store_sk#11, s_store_id#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] @@ -166,24 +166,24 @@ Condition : isnotnull(i_item_sk#14) Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] (18) BroadcastExchange Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -194,10 +194,10 @@ Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year# Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 5] +(23) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -207,17 +207,17 @@ Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand# (25) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] -(26) CometColumnarToRow [codegen id : 10] +(26) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] -(27) HashAggregate [codegen id : 10] +(27) HashAggregate [codegen id : 4] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27] Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27 AS sumsales#44] -(28) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 4] Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sumsales#44] Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] Functions [1]: [partial_sum(sumsales#44)] @@ -228,10 +228,10 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#3 Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 11] +(30) CometNativeColumnarToRow Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -(31) HashAggregate [codegen id : 11] +(31) HashAggregate [codegen id : 5] Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] Functions [1]: [sum(sumsales#44)] @@ -241,17 +241,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#3 (32) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] -(33) CometColumnarToRow [codegen id : 16] +(33) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] -(34) HashAggregate [codegen id : 16] +(34) HashAggregate [codegen id : 7] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27] Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27 AS sumsales#59] -(35) HashAggregate [codegen id : 16] +(35) HashAggregate [codegen id : 7] Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sumsales#59] Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] Functions [1]: [partial_sum(sumsales#59)] @@ -262,10 +262,10 @@ Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#5 Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 17] +(37) CometNativeColumnarToRow Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] -(38) HashAggregate [codegen id : 17] +(38) HashAggregate [codegen id : 8] Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] Functions [1]: [sum(sumsales#59)] @@ -275,17 +275,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#5 (39) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] -(40) CometColumnarToRow [codegen id : 22] +(40) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] -(41) HashAggregate [codegen id : 22] +(41) HashAggregate [codegen id : 10] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27] Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27 AS sumsales#75] -(42) HashAggregate [codegen id : 22] +(42) HashAggregate [codegen id : 10] Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sumsales#75] Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] Functions [1]: [partial_sum(sumsales#75)] @@ -296,10 +296,10 @@ Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#6 Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(44) CometColumnarToRow [codegen id : 23] +(44) CometNativeColumnarToRow Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] -(45) HashAggregate [codegen id : 23] +(45) HashAggregate [codegen id : 11] Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] Functions [1]: [sum(sumsales#75)] @@ -309,17 +309,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#6 (46) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] -(47) CometColumnarToRow [codegen id : 28] +(47) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] -(48) HashAggregate [codegen id : 28] +(48) HashAggregate [codegen id : 13] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27] Results [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27 AS sumsales#92] -(49) HashAggregate [codegen id : 28] +(49) HashAggregate [codegen id : 13] Input [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sumsales#92] Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] Functions [1]: [partial_sum(sumsales#92)] @@ -330,10 +330,10 @@ Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) CometColumnarToRow [codegen id : 29] +(51) CometNativeColumnarToRow Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] -(52) HashAggregate [codegen id : 29] +(52) HashAggregate [codegen id : 14] Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] Functions [1]: [sum(sumsales#92)] @@ -343,17 +343,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, null AS (53) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] -(54) CometColumnarToRow [codegen id : 34] +(54) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] -(55) HashAggregate [codegen id : 34] +(55) HashAggregate [codegen id : 16] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27] Results [4]: [i_category#21, i_class#20, i_brand#19, sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27 AS sumsales#110] -(56) HashAggregate [codegen id : 34] +(56) HashAggregate [codegen id : 16] Input [4]: [i_category#21, i_class#20, i_brand#19, sumsales#110] Keys [3]: [i_category#21, i_class#20, i_brand#19] Functions [1]: [partial_sum(sumsales#110)] @@ -364,10 +364,10 @@ Results [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(58) CometColumnarToRow [codegen id : 35] +(58) CometNativeColumnarToRow Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] -(59) HashAggregate [codegen id : 35] +(59) HashAggregate [codegen id : 17] Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] Keys [3]: [i_category#21, i_class#20, i_brand#19] Functions [1]: [sum(sumsales#110)] @@ -377,17 +377,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, null AS i_product_name#116, (60) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] -(61) CometColumnarToRow [codegen id : 40] +(61) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] -(62) HashAggregate [codegen id : 40] +(62) HashAggregate [codegen id : 19] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27] Results [3]: [i_category#21, i_class#20, sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27 AS sumsales#129] -(63) HashAggregate [codegen id : 40] +(63) HashAggregate [codegen id : 19] Input [3]: [i_category#21, i_class#20, sumsales#129] Keys [2]: [i_category#21, i_class#20] Functions [1]: [partial_sum(sumsales#129)] @@ -398,10 +398,10 @@ Results [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] Arguments: hashpartitioning(i_category#21, i_class#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(65) CometColumnarToRow [codegen id : 41] +(65) CometNativeColumnarToRow Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] -(66) HashAggregate [codegen id : 41] +(66) HashAggregate [codegen id : 20] Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] Keys [2]: [i_category#21, i_class#20] Functions [1]: [sum(sumsales#129)] @@ -411,17 +411,17 @@ Results [9]: [i_category#21, i_class#20, null AS i_brand#135, null AS i_product_ (67) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] -(68) CometColumnarToRow [codegen id : 46] +(68) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] -(69) HashAggregate [codegen id : 46] +(69) HashAggregate [codegen id : 22] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27] Results [2]: [i_category#21, sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27 AS sumsales#149] -(70) HashAggregate [codegen id : 46] +(70) HashAggregate [codegen id : 22] Input [2]: [i_category#21, sumsales#149] Keys [1]: [i_category#21] Functions [1]: [partial_sum(sumsales#149)] @@ -432,10 +432,10 @@ Results [3]: [i_category#21, sum#152, isEmpty#153] Input [3]: [i_category#21, sum#152, isEmpty#153] Arguments: hashpartitioning(i_category#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(72) CometColumnarToRow [codegen id : 47] +(72) CometNativeColumnarToRow Input [3]: [i_category#21, sum#152, isEmpty#153] -(73) HashAggregate [codegen id : 47] +(73) HashAggregate [codegen id : 23] Input [3]: [i_category#21, sum#152, isEmpty#153] Keys [1]: [i_category#21] Functions [1]: [sum(sumsales#149)] @@ -445,17 +445,17 @@ Results [9]: [i_category#21, null AS i_class#155, null AS i_brand#156, null AS i (74) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] -(75) CometColumnarToRow [codegen id : 52] +(75) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] -(76) HashAggregate [codegen id : 52] +(76) HashAggregate [codegen id : 25] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27] Results [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27 AS sumsales#170] -(77) HashAggregate [codegen id : 52] +(77) HashAggregate [codegen id : 25] Input [1]: [sumsales#170] Keys: [] Functions [1]: [partial_sum(sumsales#170)] @@ -466,10 +466,10 @@ Results [2]: [sum#173, isEmpty#174] Input [2]: [sum#173, isEmpty#174] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(79) CometColumnarToRow [codegen id : 53] +(79) CometNativeColumnarToRow Input [2]: [sum#173, isEmpty#174] -(80) HashAggregate [codegen id : 53] +(80) HashAggregate [codegen id : 26] Input [2]: [sum#173, isEmpty#174] Keys: [] Functions [1]: [sum(sumsales#170)] @@ -478,7 +478,7 @@ Results [9]: [null AS i_category#176, null AS i_class#177, null AS i_brand#178, (81) Union -(82) Sort [codegen id : 54] +(82) Sort [codegen id : 27] Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] Arguments: [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST], false, 0 @@ -494,7 +494,7 @@ Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumna Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] -(86) CometColumnarToRow [codegen id : 55] +(86) CometNativeColumnarToRow Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] (87) WindowGroupLimit @@ -505,7 +505,7 @@ Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 10 Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#185], [i_category#28], [sumsales#36 DESC NULLS LAST] -(89) Filter [codegen id : 56] +(89) Filter [codegen id : 28] Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] Condition : (rk#185 <= 100) @@ -517,7 +517,7 @@ Arguments: 100, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (95) -+- * CometColumnarToRow (94) ++- CometNativeColumnarToRow (94) +- CometProject (93) +- CometFilter (92) +- CometNativeScan parquet spark_catalog.default.date_dim (91) @@ -538,7 +538,7 @@ Condition : (((isnotnull(d_month_seq#186) AND (d_month_seq#186 >= 1212)) AND (d_ Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(94) CometColumnarToRow [codegen id : 1] +(94) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (95) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt index 0d338d5831..bf553c1971 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt @@ -2,14 +2,14 @@ TakeOrderedAndProject +- Filter +- Window +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- WindowGroupLimit +- Sort +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -23,31 +23,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -61,31 +61,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -99,31 +99,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -137,31 +137,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -175,31 +175,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -213,31 +213,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -251,31 +251,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -289,31 +289,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -327,22 +327,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/simplified.txt index 38fa0be072..bdb19b6fb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/simplified.txt @@ -1,147 +1,139 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (56) + WholeStageCodegen (28) Filter [rk] InputAdapter Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (55) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (54) - Sort [i_category,sumsales] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (27) + Sort [i_category,sumsales] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #11 - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #12 - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (8) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (11) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 + WholeStageCodegen (10) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (14) + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 + WholeStageCodegen (13) + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand] #10 + WholeStageCodegen (16) + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (20) + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class] #11 + WholeStageCodegen (19) + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (23) + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category] #12 + WholeStageCodegen (22) + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (26) + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #13 + WholeStageCodegen (25) + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + 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/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt index 46f670a0bd..6c26b7ae6c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt @@ -414,7 +414,7 @@ Arguments: 100, [i_category#106 ASC NULLS FIRST, i_class#107 ASC NULLS FIRST, i_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometFilter (76) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) @@ -435,7 +435,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt index b96ce25ae1..407fc9836c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -54,7 +54,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -86,7 +86,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -118,7 +118,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -150,7 +150,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -182,7 +182,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -214,7 +214,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -246,7 +246,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -278,7 +278,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt index 6096ee1c25..59865ee9ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt @@ -28,12 +28,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt index 46f670a0bd..6c26b7ae6c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt @@ -414,7 +414,7 @@ Arguments: 100, [i_category#106 ASC NULLS FIRST, i_class#107 ASC NULLS FIRST, i_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometFilter (76) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) @@ -435,7 +435,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt index b96ce25ae1..407fc9836c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -54,7 +54,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -86,7 +86,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -118,7 +118,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -150,7 +150,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -182,7 +182,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -214,7 +214,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -246,7 +246,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -278,7 +278,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6096ee1c25..59865ee9ad 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 @@ -28,12 +28,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/explain.txt index 603308d760..0d84c553c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (65) +- * Project (64) +- Window (63) - +- * CometColumnarToRow (62) + +- CometNativeColumnarToRow (62) +- CometSort (61) +- CometExchange (60) +- CometHashAggregate (59) @@ -10,7 +10,7 @@ TakeOrderedAndProject (65) +- * HashAggregate (57) +- Union (56) :- * HashAggregate (41) - : +- * CometColumnarToRow (40) + : +- CometNativeColumnarToRow (40) : +- CometColumnarExchange (39) : +- * HashAggregate (38) : +- * Project (37) @@ -24,7 +24,7 @@ TakeOrderedAndProject (65) : +- BroadcastExchange (35) : +- * Project (34) : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (9) + : :- CometNativeColumnarToRow (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (32) @@ -34,7 +34,7 @@ TakeOrderedAndProject (65) : +- WindowGroupLimit (28) : +- * Sort (27) : +- * HashAggregate (26) - : +- * CometColumnarToRow (25) + : +- CometNativeColumnarToRow (25) : +- CometColumnarExchange (24) : +- * HashAggregate (23) : +- * Project (22) @@ -45,24 +45,24 @@ TakeOrderedAndProject (65) : : : +- * ColumnarToRow (11) : : : +- Scan parquet spark_catalog.default.store_sales (10) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometProject (15) : : +- CometFilter (14) : : +- CometNativeScan parquet spark_catalog.default.store (13) : +- ReusedExchange (20) :- * HashAggregate (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometColumnarExchange (46) : +- * HashAggregate (45) : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- ReusedExchange (42) +- * HashAggregate (55) - +- * CometColumnarToRow (54) + +- CometNativeColumnarToRow (54) +- CometColumnarExchange (53) +- * HashAggregate (52) +- * HashAggregate (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- ReusedExchange (49) @@ -74,23 +74,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 70] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 5] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -105,7 +105,7 @@ ReadSchema: struct Input [3]: [s_store_sk#6, s_county#7, s_state#8] Condition : isnotnull(s_store_sk#6) -(9) CometColumnarToRow [codegen id : 7] +(9) CometNativeColumnarToRow Input [3]: [s_store_sk#6, s_county#7, s_state#8] (10) Scan parquet spark_catalog.default.store_sales @@ -116,10 +116,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Condition : isnotnull(ss_store_sk#9) @@ -138,37 +138,37 @@ Condition : isnotnull(s_store_sk#12) Input [2]: [s_store_sk#12, s_state#13] Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#14] -(16) CometColumnarToRow [codegen id : 2] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#12, s_state#14] (17) BroadcastExchange Input [2]: [s_store_sk#12, s_state#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#9] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] (20) ReusedExchange [Reuses operator id: 70] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 4] +(21) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#11] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 4] +(22) Project [codegen id : 1] Output [2]: [ss_net_profit#10, s_state#14] Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] -(23) HashAggregate [codegen id : 4] +(23) HashAggregate [codegen id : 1] Input [2]: [ss_net_profit#10, s_state#14] Keys [1]: [s_state#14] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] @@ -179,17 +179,17 @@ Results [2]: [s_state#14, sum#17] Input [2]: [s_state#14, sum#17] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(25) CometColumnarToRow [codegen id : 5] +(25) CometNativeColumnarToRow Input [2]: [s_state#14, sum#17] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 2] Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(27) Sort [codegen id : 5] +(27) Sort [codegen id : 2] Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 @@ -201,11 +201,11 @@ Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] -(30) Filter [codegen id : 6] +(30) Filter [codegen id : 3] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Condition : (ranking#20 <= 5) -(31) Project [codegen id : 6] +(31) Project [codegen id : 3] Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] @@ -213,13 +213,13 @@ Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 7] +(33) BroadcastHashJoin [codegen id : 4] Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(34) Project [codegen id : 7] +(34) Project [codegen id : 4] Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#21] Input [3]: [s_store_sk#6, s_county#7, s_state#8] @@ -227,17 +227,17 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] Input [3]: [s_store_sk#6, s_county#7, s_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 5] Output [3]: [ss_net_profit#2, s_county#7, s_state#21] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] -(38) HashAggregate [codegen id : 8] +(38) HashAggregate [codegen id : 5] Input [3]: [ss_net_profit#2, s_county#7, s_state#21] Keys [2]: [s_state#21, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] @@ -248,10 +248,10 @@ Results [3]: [s_state#21, s_county#7, sum#23] Input [3]: [s_state#21, s_county#7, sum#23] Arguments: hashpartitioning(s_state#21, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometColumnarToRow [codegen id : 9] +(40) CometNativeColumnarToRow Input [3]: [s_state#21, s_county#7, sum#23] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 6] Input [3]: [s_state#21, s_county#7, sum#23] Keys [2]: [s_state#21, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] @@ -261,17 +261,17 @@ Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as d (42) ReusedExchange [Reuses operator id: 39] Output [3]: [s_state#21, s_county#31, sum#32] -(43) CometColumnarToRow [codegen id : 18] +(43) CometNativeColumnarToRow Input [3]: [s_state#21, s_county#31, sum#32] -(44) HashAggregate [codegen id : 18] +(44) HashAggregate [codegen id : 12] Input [3]: [s_state#21, s_county#31, sum#32] Keys [2]: [s_state#21, s_county#31] Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] -(45) HashAggregate [codegen id : 18] +(45) HashAggregate [codegen id : 12] Input [2]: [total_sum#34, s_state#21] Keys [1]: [s_state#21] Functions [1]: [partial_sum(total_sum#34)] @@ -282,10 +282,10 @@ Results [3]: [s_state#21, sum#37, isEmpty#38] Input [3]: [s_state#21, sum#37, isEmpty#38] Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(47) CometColumnarToRow [codegen id : 19] +(47) CometNativeColumnarToRow Input [3]: [s_state#21, sum#37, isEmpty#38] -(48) HashAggregate [codegen id : 19] +(48) HashAggregate [codegen id : 13] Input [3]: [s_state#21, sum#37, isEmpty#38] Keys [1]: [s_state#21] Functions [1]: [sum(total_sum#34)] @@ -295,17 +295,17 @@ Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county (49) ReusedExchange [Reuses operator id: 39] Output [3]: [s_state#21, s_county#45, sum#46] -(50) CometColumnarToRow [codegen id : 28] +(50) CometNativeColumnarToRow Input [3]: [s_state#21, s_county#45, sum#46] -(51) HashAggregate [codegen id : 28] +(51) HashAggregate [codegen id : 19] Input [3]: [s_state#21, s_county#45, sum#46] Keys [2]: [s_state#21, s_county#45] Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#48] -(52) HashAggregate [codegen id : 28] +(52) HashAggregate [codegen id : 19] Input [1]: [total_sum#48] Keys: [] Functions [1]: [partial_sum(total_sum#48)] @@ -316,10 +316,10 @@ Results [2]: [sum#51, isEmpty#52] Input [2]: [sum#51, isEmpty#52] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometColumnarToRow [codegen id : 29] +(54) CometNativeColumnarToRow Input [2]: [sum#51, isEmpty#52] -(55) HashAggregate [codegen id : 29] +(55) HashAggregate [codegen id : 20] Input [2]: [sum#51, isEmpty#52] Keys: [] Functions [1]: [sum(total_sum#48)] @@ -328,7 +328,7 @@ Results [6]: [sum(total_sum#48)#53 AS total_sum#54, null AS s_state#55, null AS (56) Union -(57) HashAggregate [codegen id : 30] +(57) HashAggregate [codegen id : 21] Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] @@ -352,14 +352,14 @@ Arguments: hashpartitioning(lochierarchy#30, _w0#60, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60], [lochierarchy#30 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(62) CometColumnarToRow [codegen id : 31] +(62) CometNativeColumnarToRow Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] (63) Window Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#60, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#30, _w0#60], [total_sum#25 DESC NULLS LAST] -(64) Project [codegen id : 32] +(64) Project [codegen id : 22] Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60, rank_within_parent#61] @@ -371,7 +371,7 @@ Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometNativeScan parquet spark_catalog.default.date_dim (66) @@ -392,7 +392,7 @@ Condition : (((isnotnull(d_month_seq#62) AND (d_month_seq#62 >= 1212)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#62] Arguments: [d_date_sk#5], [d_date_sk#5] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt index d13ccdd19f..3214d035b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -9,7 +9,7 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,19 +21,19 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange @@ -43,7 +43,7 @@ TakeOrderedAndProject : +- WindowGroupLimit : +- Sort : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -55,21 +55,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -81,19 +81,19 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange @@ -103,7 +103,7 @@ TakeOrderedAndProject : +- WindowGroupLimit : +- Sort : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -115,21 +115,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -141,19 +141,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange @@ -163,7 +163,7 @@ TakeOrderedAndProject +- WindowGroupLimit +- Sort +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -175,12 +175,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/simplified.txt index ba40c1d591..fb6fc1844f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/simplified.txt @@ -1,107 +1,101 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (32) + WholeStageCodegen (22) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (31) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - 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 (30) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (9) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + 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 (21) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + InputAdapter + CometNativeColumnarToRow + CometFilter [s_store_sk,s_county,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (2) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state] #7 + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #9 - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (13) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state] #9 + WholeStageCodegen (12) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (20) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (19) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt index af71da5f87..6f9cae04fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject (66) : +- BroadcastExchange (36) : +- * Project (35) : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) + : :- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) : +- BroadcastExchange (33) @@ -121,7 +121,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -375,7 +375,7 @@ Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) @@ -396,7 +396,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt index 191d0ef18a..0ab48d6a12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -77,7 +77,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -88,7 +88,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -133,7 +133,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -144,7 +144,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt index 2832c5a348..69d6149130 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -46,8 +44,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt index af71da5f87..6f9cae04fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject (66) : +- BroadcastExchange (36) : +- * Project (35) : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) + : :- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) : +- BroadcastExchange (33) @@ -121,7 +121,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -375,7 +375,7 @@ Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) @@ -396,7 +396,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt index 191d0ef18a..0ab48d6a12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -77,7 +77,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -88,7 +88,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -133,7 +133,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -144,7 +144,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange 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 2832c5a348..69d6149130 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 @@ -31,12 +31,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -46,8 +44,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/explain.txt index 44a85d557d..c3517c4663 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (70) +CometNativeColumnarToRow (70) +- CometTakeOrderedAndProject (69) +- CometHashAggregate (68) +- CometExchange (67) @@ -30,38 +30,38 @@ : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) + : : : : : : : : : +- CometNativeColumnarToRow (6) : : : : : : : : : +- CometFilter (5) : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) + : : : : : : : : +- CometNativeColumnarToRow (12) : : : : : : : : +- CometFilter (11) : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometNativeColumnarToRow (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) + : : : : : : +- CometNativeColumnarToRow (25) : : : : : : +- CometProject (24) : : : : : : +- CometFilter (23) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) + : : : : : +- CometNativeColumnarToRow (32) : : : : : +- CometProject (31) : : : : : +- CometFilter (30) : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) : : : : +- ReusedExchange (36) : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) + : : : +- CometNativeColumnarToRow (41) : : : +- CometFilter (40) : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) + : : +- CometNativeColumnarToRow (47) : : +- CometFilter (46) : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometFilter (52) : +- CometNativeScan parquet spark_catalog.default.promotion (51) +- CometSort (63) @@ -79,10 +79,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 10] +(2) ColumnarToRow [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -(3) Filter [codegen id : 10] +(3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) @@ -98,20 +98,20 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] (13) BroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 10] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#11] Right keys [1]: [w_warehouse_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 10] +(15) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] @@ -154,20 +154,20 @@ ReadSchema: struct Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) CometColumnarToRow [codegen id : 3] +(18) CometNativeColumnarToRow Input [2]: [i_item_sk#16, i_item_desc#17] (19) BroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 10] +(20) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#4] Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(21) Project [codegen id : 10] +(21) Project [codegen id : 1] Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] @@ -186,20 +186,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) CometColumnarToRow [codegen id : 4] +(25) CometNativeColumnarToRow Input [1]: [cd_demo_sk#18] (26) BroadcastExchange Input [1]: [cd_demo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 10] +(27) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(28) Project [codegen id : 10] +(28) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] @@ -218,33 +218,33 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) CometColumnarToRow [codegen id : 5] +(32) CometNativeColumnarToRow Input [1]: [hd_demo_sk#20] (33) BroadcastExchange Input [1]: [hd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_hdemo_sk#3] Right keys [1]: [hd_demo_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 10] +(35) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] (36) ReusedExchange [Reuses operator id: 75] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(37) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#8] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(38) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] @@ -259,20 +259,20 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_week_seq#26] Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) -(41) CometColumnarToRow [codegen id : 7] +(41) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_week_seq#26] (42) BroadcastExchange Input [2]: [d_date_sk#25, d_week_seq#26] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 10] +(43) BroadcastHashJoin [codegen id : 1] Left keys [2]: [d_week_seq#24, inv_date_sk#13] Right keys [2]: [d_week_seq#26, d_date_sk#25] Join type: Inner Join condition: None -(44) Project [codegen id : 10] +(44) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] @@ -287,20 +287,20 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_date#28] Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) -(47) CometColumnarToRow [codegen id : 8] +(47) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_date#28] (48) BroadcastExchange Input [2]: [d_date_sk#27, d_date#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: (d_date#28 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(50) Project [codegen id : 1] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] @@ -315,20 +315,20 @@ ReadSchema: struct Input [1]: [p_promo_sk#29] Condition : isnotnull(p_promo_sk#29) -(53) CometColumnarToRow [codegen id : 9] +(53) CometNativeColumnarToRow Input [1]: [p_promo_sk#29] (54) BroadcastExchange Input [1]: [p_promo_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(55) BroadcastHashJoin [codegen id : 10] +(55) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_promo_sk#5] Right keys [1]: [p_promo_sk#29] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(56) Project [codegen id : 1] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] @@ -390,14 +390,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -(70) CometColumnarToRow [codegen id : 11] +(70) CometNativeColumnarToRow Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (75) -+- * CometColumnarToRow (74) ++- CometNativeColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -418,7 +418,7 @@ Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) CometColumnarToRow [codegen id : 1] +(74) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt index 081972705a..62c2b17b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -30,47 +30,47 @@ CometColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/simplified.txt index 025217a607..edcf2b00f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/simplified.txt @@ -1,107 +1,87 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - 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] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + 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] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + BroadcastExchange #6 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + BroadcastExchange #9 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + BroadcastExchange #10 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + CometNativeColumnarToRow + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt index ee3242f2b1..637427365a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt index 678f8c67df..549469c03b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt index ee3242f2b1..637427365a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 678f8c67df..549469c03b 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 @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/explain.txt index 5d2bcfdc5c..4eacb85276 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/explain.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject (79) : :- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -25,14 +25,14 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -44,14 +44,14 @@ TakeOrderedAndProject (79) : +- BroadcastExchange (56) : +- * Filter (55) : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (47) : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) + : : :- CometNativeColumnarToRow (41) : : : +- CometProject (40) : : : +- CometFilter (39) : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -62,14 +62,14 @@ TakeOrderedAndProject (79) : +- ReusedExchange (48) +- BroadcastExchange (76) +- * HashAggregate (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometColumnarExchange (73) +- * HashAggregate (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) :- * Project (68) : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) + : :- CometNativeColumnarToRow (62) : : +- CometProject (61) : : +- CometFilter (60) : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -95,7 +95,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] (5) Scan parquet spark_catalog.default.store_sales @@ -117,30 +117,30 @@ Condition : isnotnull(ss_customer_sk#8) Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] (11) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#12, d_year#13] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] @@ -151,17 +151,17 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) @@ -180,7 +180,7 @@ Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] (23) Scan parquet spark_catalog.default.store_sales @@ -191,10 +191,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Condition : isnotnull(ss_customer_sk#23) @@ -202,30 +202,30 @@ Condition : isnotnull(ss_customer_sk#23) Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#19] Right keys [1]: [ss_customer_sk#23] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#27, d_year#28] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#25] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] @@ -236,10 +236,10 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] @@ -250,7 +250,7 @@ Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_firs Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#31] Join type: Inner @@ -271,7 +271,7 @@ Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] (42) Scan parquet spark_catalog.default.web_sales @@ -282,10 +282,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Condition : isnotnull(ws_bill_customer_sk#42) @@ -293,30 +293,30 @@ Condition : isnotnull(ws_bill_customer_sk#42) Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#35] Right keys [1]: [ws_bill_customer_sk#42] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#45, d_year#46] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#44] Right keys [1]: [d_date_sk#45] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] @@ -327,17 +327,17 @@ Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum# Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#49] Results [2]: [c_customer_id#39 AS customer_id#50, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#49,17,2) AS year_total#51] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#50, year_total#51] Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) @@ -345,13 +345,13 @@ Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) Input [2]: [customer_id#50, year_total#51] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#50] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(58) Project [codegen id : 12] Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51] Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#50, year_total#51] @@ -370,7 +370,7 @@ Condition : (isnotnull(c_customer_sk#52) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] Arguments: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#54, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#55, 30, true, false, true) AS c_last_name#41] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [4]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41] (63) Scan parquet spark_catalog.default.web_sales @@ -381,10 +381,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] Condition : isnotnull(ws_bill_customer_sk#56) @@ -392,30 +392,30 @@ Condition : isnotnull(ws_bill_customer_sk#56) Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#52] Right keys [1]: [ws_bill_customer_sk#56] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58] Input [7]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] (69) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#59, d_year#60] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#58] Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58, d_date_sk#59, d_year#60] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#57))] @@ -426,10 +426,10 @@ Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum# Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] Functions [1]: [sum(UnscaledValue(ws_net_paid#57))] @@ -440,13 +440,13 @@ Results [2]: [c_customer_id#39 AS customer_id#63, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#63, year_total#64] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#63] Join type: Inner Join condition: (CASE WHEN (year_total#51 > 0.00) THEN (year_total#64 / year_total#51) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) -(78) Project [codegen id : 16] +(78) Project [codegen id : 12] Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51, customer_id#63, year_total#64] @@ -458,7 +458,7 @@ Arguments: 100, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULL Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometFilter (81) +- CometNativeScan parquet spark_catalog.default.date_dim (80) @@ -474,7 +474,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (83) BroadcastExchange @@ -483,7 +483,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (87) -+- * CometColumnarToRow (86) ++- CometNativeColumnarToRow (86) +- CometFilter (85) +- CometNativeScan parquet spark_catalog.default.date_dim (84) @@ -499,7 +499,7 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_year#28] Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) -(86) CometColumnarToRow [codegen id : 1] +(86) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_year#28] (87) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt index 0d59f7b833..5d218998be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -23,23 +23,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -49,24 +49,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -76,19 +76,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -98,7 +98,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/simplified.txt index 9a8a029e1b..95bb7acab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -7,17 +7,17 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] @@ -30,69 +30,65 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -102,25 +98,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt index b0ae8bb695..80326e1f49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_year#26] Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_year#26] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt index 72e39422e0..9ec698fdca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt index b0ae8bb695..80326e1f49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_year#26] Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_year#26] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 72e39422e0..9ec698fdca 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 @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/explain.txt index eddee4d8e5..1c2b04e6b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (127) +CometNativeColumnarToRow (127) +- CometTakeOrderedAndProject (126) +- CometProject (125) +- CometSortMergeJoin (124) @@ -25,7 +25,7 @@ : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -136,10 +136,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Condition : isnotnull(cs_item_sk#1) @@ -158,33 +158,33 @@ Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarc Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (8) BroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (11) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#13, d_year#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] @@ -236,36 +236,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(24) ColumnarToRow [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) (26) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#22] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] (29) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#32, d_year#33] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] @@ -317,36 +317,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 9] +(42) ColumnarToRow [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -(43) Filter [codegen id : 9] +(43) Filter [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Condition : isnotnull(ws_item_sk#41) (44) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -(45) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_item_sk#41] Right keys [1]: [i_item_sk#46] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(46) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] (47) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#51, d_year#52] -(48) BroadcastHashJoin [codegen id : 9] +(48) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#45] Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(49) Project [codegen id : 9] +(49) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] @@ -443,36 +443,36 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 12] +(70) ColumnarToRow [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -(71) Filter [codegen id : 12] +(71) Filter [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] Condition : isnotnull(cs_item_sk#64) (72) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -(73) BroadcastHashJoin [codegen id : 12] +(73) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_item_sk#64] Right keys [1]: [i_item_sk#70] Join type: Inner Join condition: None -(74) Project [codegen id : 12] +(74) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] (75) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#75, d_year#76] -(76) BroadcastHashJoin [codegen id : 12] +(76) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#68] Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(77) Project [codegen id : 12] +(77) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] @@ -508,36 +508,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 15] +(85) ColumnarToRow [codegen id : 5] Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -(86) Filter [codegen id : 15] +(86) Filter [codegen id : 5] Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] Condition : isnotnull(ss_item_sk#81) (87) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] -(88) BroadcastHashJoin [codegen id : 15] +(88) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#81] Right keys [1]: [i_item_sk#86] Join type: Inner Join condition: None -(89) Project [codegen id : 15] +(89) Project [codegen id : 5] Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] Input [10]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] (90) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#91, d_year#92] -(91) BroadcastHashJoin [codegen id : 15] +(91) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#85] Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(92) Project [codegen id : 15] +(92) Project [codegen id : 5] Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] Input [11]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_date_sk#91, d_year#92] @@ -573,36 +573,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 18] +(100) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -(101) Filter [codegen id : 18] +(101) Filter [codegen id : 6] Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] Condition : isnotnull(ws_item_sk#97) (102) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] -(103) BroadcastHashJoin [codegen id : 18] +(103) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#97] Right keys [1]: [i_item_sk#102] Join type: Inner Join condition: None -(104) Project [codegen id : 18] +(104) Project [codegen id : 6] Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] Input [10]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] (105) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#107, d_year#108] -(106) BroadcastHashJoin [codegen id : 18] +(106) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#101] Right keys [1]: [d_date_sk#107] Join type: Inner Join condition: None -(107) Project [codegen id : 18] +(107) Project [codegen id : 6] Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] Input [11]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_date_sk#107, d_year#108] @@ -688,14 +688,14 @@ Arguments: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#120 ASC NULLS FIRST,sales_amt_diff#121 ASC NULLS FIRST], output=[prev_year#116,year#117,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#118,curr_yr_cnt#119,sales_cnt_diff#120,sales_amt_diff#121]), [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], 100, 0, [sales_cnt_diff#120 ASC NULLS FIRST, sales_amt_diff#121 ASC NULLS FIRST], [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] -(127) CometColumnarToRow [codegen id : 19] +(127) CometNativeColumnarToRow Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (131) -+- * CometColumnarToRow (130) ++- CometNativeColumnarToRow (130) +- CometFilter (129) +- CometNativeScan parquet spark_catalog.default.date_dim (128) @@ -711,7 +711,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (131) BroadcastExchange @@ -724,7 +724,7 @@ Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN d Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 BroadcastExchange (135) -+- * CometColumnarToRow (134) ++- CometNativeColumnarToRow (134) +- CometFilter (133) +- CometNativeScan parquet spark_catalog.default.date_dim (132) @@ -740,7 +740,7 @@ ReadSchema: struct Input [2]: [d_date_sk#75, d_year#76] Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) -(134) CometColumnarToRow [codegen id : 1] +(134) CometNativeColumnarToRow Input [2]: [d_date_sk#75, d_year#76] (135) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt index 9f6954be38..42106a0ffd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometSortMergeJoin @@ -25,16 +25,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -55,12 +55,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -81,12 +81,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -117,16 +117,16 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -147,12 +147,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -173,12 +173,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/simplified.txt index aaf98f4b09..84b015b466 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/simplified.txt @@ -1,173 +1,165 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - 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 [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 [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] +CometNativeColumnarToRow + CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + 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 [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 [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] - 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] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometExchange [cr_order_number,cr_item_sk] #7 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (2) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometExchange [sr_ticket_number,sr_item_sk] #9 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (3) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometExchange [wr_order_number,wr_item_sk] #11 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 + 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] #14 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] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (4) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (5) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (6) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt index 6925eb301e..e66f444aba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt index d0520c8b63..0e2370eea4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt index 6925eb301e..e66f444aba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange 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 d0520c8b63..0e2370eea4 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 @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/explain.txt index 744f3a986d..35932328b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/explain.txt @@ -1,19 +1,19 @@ == Physical Plan == -* CometColumnarToRow (110) +CometNativeColumnarToRow (110) +- CometTakeOrderedAndProject (109) +- CometHashAggregate (108) +- CometColumnarExchange (107) +- * HashAggregate (106) +- Union (105) :- * HashAggregate (90) - : +- * CometColumnarToRow (89) + : +- CometNativeColumnarToRow (89) : +- CometColumnarExchange (88) : +- * HashAggregate (87) : +- Union (86) : :- * Project (32) : : +- * BroadcastHashJoin LeftOuter BuildRight (31) : : :- * HashAggregate (16) - : : : +- * CometColumnarToRow (15) + : : : +- CometNativeColumnarToRow (15) : : : +- CometColumnarExchange (14) : : : +- * HashAggregate (13) : : : +- * Project (12) @@ -25,12 +25,12 @@ : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : +- ReusedExchange (4) : : : +- BroadcastExchange (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometFilter (8) : : : +- CometNativeScan parquet spark_catalog.default.store (7) : : +- BroadcastExchange (30) : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) + : : +- CometNativeColumnarToRow (28) : : +- CometColumnarExchange (27) : : +- * HashAggregate (26) : : +- * Project (25) @@ -46,7 +46,7 @@ : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) : : :- BroadcastExchange (42) : : : +- * HashAggregate (41) - : : : +- * CometColumnarToRow (40) + : : : +- CometNativeColumnarToRow (40) : : : +- CometColumnarExchange (39) : : : +- * HashAggregate (38) : : : +- * Project (37) @@ -55,7 +55,7 @@ : : : : +- Scan parquet spark_catalog.default.catalog_sales (33) : : : +- ReusedExchange (35) : : +- * HashAggregate (51) - : : +- * CometColumnarToRow (50) + : : +- CometNativeColumnarToRow (50) : : +- CometColumnarExchange (49) : : +- * HashAggregate (48) : : +- * Project (47) @@ -66,7 +66,7 @@ : +- * Project (85) : +- * BroadcastHashJoin LeftOuter BuildRight (84) : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) + : : +- CometNativeColumnarToRow (68) : : +- CometColumnarExchange (67) : : +- * HashAggregate (66) : : +- * Project (65) @@ -78,12 +78,12 @@ : : : : +- Scan parquet spark_catalog.default.web_sales (54) : : : +- ReusedExchange (57) : : +- BroadcastExchange (63) - : : +- * CometColumnarToRow (62) + : : +- CometNativeColumnarToRow (62) : : +- CometFilter (61) : : +- CometNativeScan parquet spark_catalog.default.web_page (60) : +- BroadcastExchange (83) : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) + : +- CometNativeColumnarToRow (81) : +- CometColumnarExchange (80) : +- * HashAggregate (79) : +- * Project (78) @@ -96,18 +96,18 @@ : : +- ReusedExchange (73) : +- ReusedExchange (76) :- * HashAggregate (97) - : +- * CometColumnarToRow (96) + : +- CometNativeColumnarToRow (96) : +- CometColumnarExchange (95) : +- * HashAggregate (94) : +- * HashAggregate (93) - : +- * CometColumnarToRow (92) + : +- CometNativeColumnarToRow (92) : +- ReusedExchange (91) +- * HashAggregate (104) - +- * CometColumnarToRow (103) + +- CometNativeColumnarToRow (103) +- CometColumnarExchange (102) +- * HashAggregate (101) +- * HashAggregate (100) - +- * CometColumnarToRow (99) + +- CometNativeColumnarToRow (99) +- ReusedExchange (98) @@ -119,23 +119,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] @@ -150,24 +150,24 @@ ReadSchema: struct Input [1]: [s_store_sk#7] Condition : isnotnull(s_store_sk#7) -(9) CometColumnarToRow [codegen id : 2] +(9) CometNativeColumnarToRow Input [1]: [s_store_sk#7] (10) BroadcastExchange Input [1]: [s_store_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(12) Project [codegen id : 3] +(12) Project [codegen id : 1] Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Keys [1]: [s_store_sk#7] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] @@ -178,10 +178,10 @@ Results [3]: [s_store_sk#7, sum#10, sum#11] Input [3]: [s_store_sk#7, sum#10, sum#11] Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(15) CometColumnarToRow [codegen id : 8] +(15) CometNativeColumnarToRow Input [3]: [s_store_sk#7, sum#10, sum#11] -(16) HashAggregate [codegen id : 8] +(16) HashAggregate [codegen id : 4] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] @@ -196,40 +196,40 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Condition : isnotnull(sr_store_sk#16) (20) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#20] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] (23) ReusedExchange [Reuses operator id: 10] Output [1]: [s_store_sk#21] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_store_sk#16] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Keys [1]: [s_store_sk#21] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] @@ -240,10 +240,10 @@ Results [3]: [s_store_sk#21, sum#24, sum#25] Input [3]: [s_store_sk#21, sum#24, sum#25] Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#21, sum#24, sum#25] -(29) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 3] Input [3]: [s_store_sk#21, sum#24, sum#25] Keys [1]: [s_store_sk#21] Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] @@ -254,13 +254,13 @@ Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26 Input [3]: [s_store_sk#21, returns#28, profit_loss#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#7] Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 4] Output [5]: [store channel AS channel#30, s_store_sk#7 AS id#31, sales#14, coalesce(returns#28, 0.00) AS returns#32, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#33] Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] @@ -271,23 +271,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 5] Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] (35) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#38] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#37] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 5] Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] -(38) HashAggregate [codegen id : 10] +(38) HashAggregate [codegen id : 5] Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Keys [1]: [cs_call_center_sk#34] Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] @@ -298,10 +298,10 @@ Results [3]: [cs_call_center_sk#34, sum#41, sum#42] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometColumnarToRow [codegen id : 11] +(40) CometNativeColumnarToRow Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 6] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Keys [1]: [cs_call_center_sk#34] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] @@ -319,23 +319,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] +(44) ColumnarToRow [codegen id : 7] Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] (45) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#50] -(46) BroadcastHashJoin [codegen id : 13] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cr_returned_date_sk#49] Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(47) Project [codegen id : 13] +(47) Project [codegen id : 7] Output [2]: [cr_return_amount#47, cr_net_loss#48] Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] -(48) HashAggregate [codegen id : 13] +(48) HashAggregate [codegen id : 7] Input [2]: [cr_return_amount#47, cr_net_loss#48] Keys: [] Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] @@ -346,7 +346,7 @@ Results [2]: [sum#53, sum#54] Input [2]: [sum#53, sum#54] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) CometColumnarToRow +(50) CometNativeColumnarToRow Input [2]: [sum#53, sum#54] (51) HashAggregate @@ -356,11 +356,11 @@ Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_ne Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] -(52) BroadcastNestedLoopJoin [codegen id : 14] +(52) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 8] Output [5]: [catalog channel AS channel#59, cs_call_center_sk#34 AS id#60, sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#61] Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] @@ -372,23 +372,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 17] +(55) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -(56) Filter [codegen id : 17] +(56) Filter [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] Condition : isnotnull(ws_web_page_sk#62) (57) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#66] -(58) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#65] Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(59) Project [codegen id : 9] Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] @@ -403,24 +403,24 @@ ReadSchema: struct Input [1]: [wp_web_page_sk#67] Condition : isnotnull(wp_web_page_sk#67) -(62) CometColumnarToRow [codegen id : 16] +(62) CometNativeColumnarToRow Input [1]: [wp_web_page_sk#67] (63) BroadcastExchange Input [1]: [wp_web_page_sk#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(64) BroadcastHashJoin [codegen id : 17] +(64) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_web_page_sk#62] Right keys [1]: [wp_web_page_sk#67] Join type: Inner Join condition: None -(65) Project [codegen id : 17] +(65) Project [codegen id : 9] Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 9] Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Keys [1]: [wp_web_page_sk#67] Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] @@ -431,10 +431,10 @@ Results [3]: [wp_web_page_sk#67, sum#70, sum#71] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometColumnarToRow [codegen id : 22] +(68) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -(69) HashAggregate [codegen id : 22] +(69) HashAggregate [codegen id : 12] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Keys [1]: [wp_web_page_sk#67] Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] @@ -449,40 +449,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -(72) Filter [codegen id : 20] +(72) Filter [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] Condition : isnotnull(wr_web_page_sk#76) (73) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#80] -(74) BroadcastHashJoin [codegen id : 20] +(74) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_returned_date_sk#79] Right keys [1]: [d_date_sk#80] Join type: Inner Join condition: None -(75) Project [codegen id : 20] +(75) Project [codegen id : 10] Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] (76) ReusedExchange [Reuses operator id: 63] Output [1]: [wp_web_page_sk#81] -(77) BroadcastHashJoin [codegen id : 20] +(77) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_web_page_sk#76] Right keys [1]: [wp_web_page_sk#81] Join type: Inner Join condition: None -(78) Project [codegen id : 20] +(78) Project [codegen id : 10] Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -(79) HashAggregate [codegen id : 20] +(79) HashAggregate [codegen id : 10] Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Keys [1]: [wp_web_page_sk#81] Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] @@ -493,10 +493,10 @@ Results [3]: [wp_web_page_sk#81, sum#84, sum#85] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(81) CometColumnarToRow [codegen id : 21] +(81) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -(82) HashAggregate [codegen id : 21] +(82) HashAggregate [codegen id : 11] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Keys [1]: [wp_web_page_sk#81] Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] @@ -507,19 +507,19 @@ Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77) Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 22] +(84) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wp_web_page_sk#67] Right keys [1]: [wp_web_page_sk#81] Join type: LeftOuter Join condition: None -(85) Project [codegen id : 22] +(85) Project [codegen id : 12] Output [5]: [web channel AS channel#90, wp_web_page_sk#67 AS id#91, sales#74, coalesce(returns#88, 0.00) AS returns#92, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#93] Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] (86) Union -(87) HashAggregate [codegen id : 23] +(87) HashAggregate [codegen id : 13] Input [5]: [channel#30, id#31, sales#14, returns#32, profit#33] Keys [2]: [channel#30, id#31] Functions [3]: [partial_sum(sales#14), partial_sum(returns#32), partial_sum(profit#33)] @@ -530,10 +530,10 @@ Results [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(89) CometColumnarToRow [codegen id : 24] +(89) CometNativeColumnarToRow Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(90) HashAggregate [codegen id : 24] +(90) HashAggregate [codegen id : 14] Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Keys [2]: [channel#30, id#31] Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] @@ -543,17 +543,17 @@ Results [5]: [channel#30, id#31, cast(sum(sales#14)#106 as decimal(37,2)) AS sal (91) ReusedExchange [Reuses operator id: 88] Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(92) CometColumnarToRow [codegen id : 48] +(92) CometNativeColumnarToRow Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(93) HashAggregate [codegen id : 48] +(93) HashAggregate [codegen id : 28] Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Keys [2]: [channel#30, id#31] Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] Results [4]: [channel#30, sum(sales#14)#106 AS sales#112, sum(returns#32)#107 AS returns#113, sum(profit#33)#108 AS profit#114] -(94) HashAggregate [codegen id : 48] +(94) HashAggregate [codegen id : 28] Input [4]: [channel#30, sales#112, returns#113, profit#114] Keys [1]: [channel#30] Functions [3]: [partial_sum(sales#112), partial_sum(returns#113), partial_sum(profit#114)] @@ -564,10 +564,10 @@ Results [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, i Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(96) CometColumnarToRow [codegen id : 49] +(96) CometNativeColumnarToRow Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -(97) HashAggregate [codegen id : 49] +(97) HashAggregate [codegen id : 29] Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] Keys [1]: [channel#30] Functions [3]: [sum(sales#112), sum(returns#113), sum(profit#114)] @@ -577,17 +577,17 @@ Results [5]: [channel#30, null AS id#130, sum(sales#112)#127 AS sales#131, sum(r (98) ReusedExchange [Reuses operator id: 88] Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(99) CometColumnarToRow [codegen id : 73] +(99) CometNativeColumnarToRow Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(100) HashAggregate [codegen id : 73] +(100) HashAggregate [codegen id : 43] Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Keys [2]: [channel#30, id#31] Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] Results [3]: [sum(sales#14)#106 AS sales#134, sum(returns#32)#107 AS returns#135, sum(profit#33)#108 AS profit#136] -(101) HashAggregate [codegen id : 73] +(101) HashAggregate [codegen id : 43] Input [3]: [sales#134, returns#135, profit#136] Keys: [] Functions [3]: [partial_sum(sales#134), partial_sum(returns#135), partial_sum(profit#136)] @@ -598,10 +598,10 @@ Results [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(103) CometColumnarToRow [codegen id : 74] +(103) CometNativeColumnarToRow Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -(104) HashAggregate [codegen id : 74] +(104) HashAggregate [codegen id : 44] Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Keys: [] Functions [3]: [sum(sales#134), sum(returns#135), sum(profit#136)] @@ -610,7 +610,7 @@ Results [5]: [null AS channel#152, null AS id#153, sum(sales#134)#149 AS sales#1 (105) Union -(106) HashAggregate [codegen id : 75] +(106) HashAggregate [codegen id : 45] Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Functions: [] @@ -630,14 +630,14 @@ Functions: [] Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#30 ASC NULLS FIRST,id#31 ASC NULLS FIRST], output=[channel#30,id#31,sales#109,returns#110,profit#111]), [channel#30, id#31, sales#109, returns#110, profit#111], 100, 0, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30, id#31, sales#109, returns#110, profit#111] -(110) CometColumnarToRow [codegen id : 76] +(110) CometNativeColumnarToRow Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (115) -+- * CometColumnarToRow (114) ++- CometNativeColumnarToRow (114) +- CometProject (113) +- CometFilter (112) +- CometNativeScan parquet spark_catalog.default.date_dim (111) @@ -658,7 +658,7 @@ Condition : (((isnotnull(d_date#157) AND (d_date#157 >= 1998-08-04)) AND (d_date Input [2]: [d_date_sk#6, d_date#157] Arguments: [d_date_sk#6], [d_date_sk#6] -(114) CometColumnarToRow [codegen id : 1] +(114) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (115) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt index 81b485e6b3..178b946ecb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt @@ -1,18 +1,18 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Project : : +- BroadcastHashJoin : : :- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -24,22 +24,22 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -51,19 +51,19 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -72,12 +72,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -86,14 +86,14 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -105,17 +105,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_page : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -127,27 +127,27 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Project : : +- BroadcastHashJoin : : :- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -159,22 +159,22 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -186,19 +186,19 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -207,12 +207,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -221,14 +221,14 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -240,17 +240,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_page : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -262,27 +262,27 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -294,22 +294,22 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -321,19 +321,19 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -342,12 +342,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -356,14 +356,14 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- BroadcastHashJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -375,17 +375,17 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -397,12 +397,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/simplified.txt index 6f40fcdddf..9a9dde3ef9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/simplified.txt @@ -1,180 +1,172 @@ -WholeStageCodegen (76) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (24) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (45) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (14) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (13) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_store_sk] #3 + WholeStageCodegen (1) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_store_sk] #7 + WholeStageCodegen (2) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #9 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow + InputAdapter + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (8) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow + CometNativeColumnarToRow + CometColumnarExchange [cs_call_center_sk] #9 + WholeStageCodegen (5) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (7) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (12) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [wp_web_page_sk] #11 + WholeStageCodegen (9) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_web_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow + InputAdapter + BroadcastExchange #12 + CometNativeColumnarToRow + CometFilter [wp_web_page_sk] + CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (11) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - CometColumnarExchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + CometNativeColumnarToRow + CometColumnarExchange [wp_web_page_sk] #14 + WholeStageCodegen (10) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Filter [wr_web_page_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #15 - WholeStageCodegen (48) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #16 - WholeStageCodegen (73) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [wp_web_page_sk] #12 + WholeStageCodegen (29) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel] #15 + WholeStageCodegen (28) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (44) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #16 + WholeStageCodegen (43) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt index 4b5efc84e3..06923495a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt @@ -580,7 +580,7 @@ Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -601,7 +601,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt index b4318d03e2..40e2c31863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt @@ -23,7 +23,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -257,7 +257,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt index ac3d312ee8..d559f4bdcb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt @@ -33,12 +33,10 @@ WholeStageCodegen (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt index 4b5efc84e3..06923495a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt @@ -580,7 +580,7 @@ Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -601,7 +601,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt index b4318d03e2..40e2c31863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt @@ -23,7 +23,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -257,7 +257,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ac3d312ee8..d559f4bdcb 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 @@ -33,12 +33,10 @@ WholeStageCodegen (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/explain.txt index dbd053cfac..e227adc462 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (70) +- * Project (69) - +- * CometColumnarToRow (68) + +- CometNativeColumnarToRow (68) +- CometSortMergeJoin (67) :- CometProject (45) : +- CometSortMergeJoin (44) @@ -372,7 +372,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner -(68) CometColumnarToRow [codegen id : 4] +(68) CometNativeColumnarToRow Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] (69) Project [codegen id : 4] @@ -387,7 +387,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -403,7 +403,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (74) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt index 65256f2124..af4b7000b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSortMergeJoin :- CometProject : +- CometSortMergeJoin @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/simplified.txt index 4711669981..e4f8f19efa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (4) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] @@ -24,11 +24,9 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt index 82a5357f89..8e3014f399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt index 273db28e77..ef0945f5d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt @@ -21,11 +21,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.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt index 82a5357f89..8e3014f399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort 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 273db28e77..ef0945f5d6 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 @@ -21,11 +21,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.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/explain.txt index ae7a6e67aa..6f441274cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (119) +CometNativeColumnarToRow (119) +- CometTakeOrderedAndProject (118) +- CometHashAggregate (117) +- CometExchange (116) @@ -653,14 +653,14 @@ Functions: [] Input [5]: [channel#74, id#75, sales#113, returns#114, profit#115] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#74 ASC NULLS FIRST,id#75 ASC NULLS FIRST], output=[channel#74,id#75,sales#113,returns#114,profit#115]), [channel#74, id#75, sales#113, returns#114, profit#115], 100, 0, [channel#74 ASC NULLS FIRST, id#75 ASC NULLS FIRST], [channel#74, id#75, sales#113, returns#114, profit#115] -(119) CometColumnarToRow [codegen id : 10] +(119) CometNativeColumnarToRow Input [5]: [channel#74, id#75, sales#113, returns#114, profit#115] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometNativeScan parquet spark_catalog.default.date_dim (120) @@ -681,7 +681,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (124) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt index f752943c47..da58e863bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -28,7 +28,7 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,7 +160,7 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -292,7 +292,7 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/simplified.txt index cfce9e20a1..37dfa17b65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/simplified.txt @@ -1,137 +1,133 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometExchange [channel,id,sales,returns,profit] #1 CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - 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] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [cp_catalog_page_id] #11 - 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] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #12 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - 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] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #16 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [s_store_id] #3 + 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] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 + WholeStageCodegen (1) + Filter [ss_store_sk,ss_item_sk,ss_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #8 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [cp_catalog_page_id] #11 + 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] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #12 + WholeStageCodegen (2) + Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [web_site_id] #15 + 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] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #16 + WholeStageCodegen (3) + Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #19 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange #20 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt index f9901f13d7..9c725ce21e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt @@ -648,7 +648,7 @@ Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -669,7 +669,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt index 0d6844d93c..943d6e8d1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,7 +156,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -285,7 +285,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt index d27ab827e1..dce81e1279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #6 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt index f9901f13d7..9c725ce21e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt @@ -648,7 +648,7 @@ Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -669,7 +669,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt index 0d6844d93c..943d6e8d1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,7 +156,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -285,7 +285,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d27ab827e1..dce81e1279 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 @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #6 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/explain.txt index 03194c1609..4611198fe1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (41) +- * Project (40) +- Window (39) - +- * CometColumnarToRow (38) + +- CometNativeColumnarToRow (38) +- CometSort (37) +- CometExchange (36) +- CometHashAggregate (35) @@ -10,7 +10,7 @@ TakeOrderedAndProject (41) +- * HashAggregate (33) +- Union (32) :- * HashAggregate (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometColumnarExchange (15) : +- * HashAggregate (14) : +- * Project (13) @@ -22,23 +22,23 @@ TakeOrderedAndProject (41) : : : +- Scan parquet spark_catalog.default.web_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) + : +- CometNativeColumnarToRow (19) : +- ReusedExchange (18) +- * HashAggregate (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * HashAggregate (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- ReusedExchange (25) @@ -50,23 +50,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ws_item_sk#1, ws_net_paid#2] Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] @@ -85,24 +85,24 @@ Condition : isnotnull(i_item_sk#6) Input [3]: [i_item_sk#6, i_class#7, i_category#8] Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#6, i_class#9, i_category#10] (11) BroadcastExchange Input [3]: [i_item_sk#6, i_class#9, i_category#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [ws_net_paid#2, i_class#9, i_category#10] Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [ws_net_paid#2, i_class#9, i_category#10] Keys [2]: [i_category#10, i_class#9] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] @@ -113,10 +113,10 @@ Results [3]: [i_category#10, i_class#9, sum#12] Input [3]: [i_category#10, i_class#9, sum#12] Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [3]: [i_category#10, i_class#9, sum#12] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [3]: [i_category#10, i_class#9, sum#12] Keys [2]: [i_category#10, i_class#9] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] @@ -126,17 +126,17 @@ Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as dec (18) ReusedExchange [Reuses operator id: 15] Output [3]: [i_category#10, i_class#9, sum#20] -(19) CometColumnarToRow [codegen id : 8] +(19) CometNativeColumnarToRow Input [3]: [i_category#10, i_class#9, sum#20] -(20) HashAggregate [codegen id : 8] +(20) HashAggregate [codegen id : 4] Input [3]: [i_category#10, i_class#9, sum#20] Keys [2]: [i_category#10, i_class#9] Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#13] Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#13,17,2) AS total_sum#22, i_category#10] -(21) HashAggregate [codegen id : 8] +(21) HashAggregate [codegen id : 4] Input [2]: [total_sum#22, i_category#10] Keys [1]: [i_category#10] Functions [1]: [partial_sum(total_sum#22)] @@ -147,10 +147,10 @@ Results [3]: [i_category#10, sum#25, isEmpty#26] Input [3]: [i_category#10, sum#25, isEmpty#26] Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 9] +(23) CometNativeColumnarToRow Input [3]: [i_category#10, sum#25, isEmpty#26] -(24) HashAggregate [codegen id : 9] +(24) HashAggregate [codegen id : 5] Input [3]: [i_category#10, sum#25, isEmpty#26] Keys [1]: [i_category#10] Functions [1]: [sum(total_sum#22)] @@ -160,17 +160,17 @@ Results [6]: [sum(total_sum#22)#27 AS total_sum#28, i_category#10, null AS i_cla (25) ReusedExchange [Reuses operator id: 15] Output [3]: [i_category#10, i_class#9, sum#33] -(26) CometColumnarToRow [codegen id : 13] +(26) CometNativeColumnarToRow Input [3]: [i_category#10, i_class#9, sum#33] -(27) HashAggregate [codegen id : 13] +(27) HashAggregate [codegen id : 7] Input [3]: [i_category#10, i_class#9, sum#33] Keys [2]: [i_category#10, i_class#9] Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#13] Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#13,17,2) AS total_sum#35] -(28) HashAggregate [codegen id : 13] +(28) HashAggregate [codegen id : 7] Input [1]: [total_sum#35] Keys: [] Functions [1]: [partial_sum(total_sum#35)] @@ -181,10 +181,10 @@ Results [2]: [sum#38, isEmpty#39] Input [2]: [sum#38, isEmpty#39] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 14] +(30) CometNativeColumnarToRow Input [2]: [sum#38, isEmpty#39] -(31) HashAggregate [codegen id : 14] +(31) HashAggregate [codegen id : 8] Input [2]: [sum#38, isEmpty#39] Keys: [] Functions [1]: [sum(total_sum#35)] @@ -193,7 +193,7 @@ Results [6]: [sum(total_sum#35)#40 AS total_sum#41, null AS i_category#42, null (32) Union -(33) HashAggregate [codegen id : 15] +(33) HashAggregate [codegen id : 9] Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] Functions: [] @@ -217,14 +217,14 @@ Arguments: hashpartitioning(lochierarchy#19, _w0#47, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47], [lochierarchy#19 ASC NULLS FIRST, _w0#47 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] -(38) CometColumnarToRow [codegen id : 16] +(38) CometNativeColumnarToRow Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] (39) Window Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#47, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#48], [lochierarchy#19, _w0#47], [total_sum#14 DESC NULLS LAST] -(40) Project [codegen id : 17] +(40) Project [codegen id : 10] Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47, rank_within_parent#48] @@ -236,7 +236,7 @@ Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometNativeScan parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= 1212)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#49] Arguments: [d_date_sk#5], [d_date_sk#5] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt index a5b37c422c..d56e25c3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -9,7 +9,7 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,26 +21,26 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -52,26 +52,26 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -83,17 +83,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/simplified.txt index 2ccc8c0c39..ad1b79bc92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/simplified.txt @@ -1,71 +1,65 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (17) + WholeStageCodegen (10) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - 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 (15) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + 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 (9) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #6 - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #7 - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (5) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category] #6 + WholeStageCodegen (4) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (8) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #7 + WholeStageCodegen (7) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt index b623de85d0..517f7f9c22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt @@ -210,7 +210,7 @@ Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -231,7 +231,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt index 46c47555a8..13ff1e34cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt index 11e3c03d80..67ea1dbf75 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt @@ -23,12 +23,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt index b623de85d0..517f7f9c22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt @@ -210,7 +210,7 @@ Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -231,7 +231,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt index 46c47555a8..13ff1e34cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 11e3c03d80..67ea1dbf75 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 @@ -23,12 +23,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/explain.txt index 646b85d12e..04452de0b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -* CometColumnarToRow (25) +CometNativeColumnarToRow (25) +- CometSort (24) +- CometColumnarExchange (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -19,7 +19,7 @@ : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.store_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -34,10 +34,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) @@ -56,37 +56,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -97,10 +97,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -115,14 +115,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] @@ -134,14 +134,14 @@ Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 7] +(25) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.date_dim (26) @@ -162,7 +162,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt index 9ab057d467..b8d60879d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt @@ -1,13 +1,13 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -19,17 +19,17 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/simplified.txt index c1ec019e57..efcfa8e8f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/simplified.txt @@ -1,47 +1,39 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] +CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt index 96a0ec392e..ee4192a1eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt @@ -142,7 +142,7 @@ Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt index 3dbaf2e346..91e569a85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt index 46191f59cd..7adce04170 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt @@ -23,12 +23,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt index 96a0ec392e..ee4192a1eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt @@ -142,7 +142,7 @@ Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt index 3dbaf2e346..91e569a85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 46191f59cd..7adce04170 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 @@ -23,12 +23,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/explain.txt index 7cfe036ae9..6cc813a023 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -11,7 +11,7 @@ TakeOrderedAndProject (43) : :- * Project (24) : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) + : : : :- CometNativeColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : +- BroadcastExchange (9) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- Scan parquet spark_catalog.default.catalog_sales (16) : : +- ReusedExchange (18) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer_address (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) @@ -55,7 +55,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] (6) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] @@ -85,7 +85,7 @@ Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi @@ -98,19 +98,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] (13) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#10] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#8 AS customer_sk#11] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] @@ -121,19 +121,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] +(17) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] (18) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#14] -(19) BroadcastHashJoin [codegen id : 6] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(20) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] @@ -143,13 +143,13 @@ Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] Input [1]: [customer_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#11] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 9] +(24) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -168,20 +168,20 @@ Condition : (ca_county#17 IN (Walker County,Richland County,Gaines County,Dougla Input [2]: [ca_address_sk#16, ca_county#17] Arguments: [ca_address_sk#16], [ca_address_sk#16] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [1]: [ca_address_sk#16] (29) BroadcastExchange Input [1]: [ca_address_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(31) Project [codegen id : 9] +(31) Project [codegen id : 4] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] @@ -200,24 +200,24 @@ Condition : isnotnull(cd_demo_sk#18) Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#19, 1)) AS cd_gender#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#29, cd_purchase_estimate#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#23, 10)) AS cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(35) CometColumnarToRow [codegen id : 8] +(35) CometNativeColumnarToRow Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] (36) BroadcastExchange Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 4] Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(39) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 4] Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [partial_count(1)] @@ -228,10 +228,10 @@ Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_pur Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 5] Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [1]: [count(1)] @@ -246,7 +246,7 @@ Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FI Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -267,7 +267,7 @@ Condition : (((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = Input [3]: [d_date_sk#7, d_year#40, d_moy#41] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt index 8aebe6be9a..9091536e96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : :- Project : : +- BroadcastHashJoin : : :- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange @@ -20,12 +20,12 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,7 +37,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -47,17 +47,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/simplified.txt index 11bfb6f6c3..5b1aad4ef8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,13 +13,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -27,18 +27,16 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 Union - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -47,7 +45,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -58,17 +56,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/explain.txt index 7e11c91157..7fc8ce4b83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/explain.txt @@ -232,7 +232,7 @@ Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36 Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/simplified.txt index 38f41d10a5..b54bf7704a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt index 7e11c91157..7fc8ce4b83 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt @@ -232,7 +232,7 @@ Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36 Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 38f41d10a5..b54bf7704a 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/explain.txt index 700f90222d..43ee0bebbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/explain.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject (79) : :- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -25,14 +25,14 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -44,14 +44,14 @@ TakeOrderedAndProject (79) : +- BroadcastExchange (56) : +- * Filter (55) : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (47) : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) + : : :- CometNativeColumnarToRow (41) : : : +- CometProject (40) : : : +- CometFilter (39) : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -62,14 +62,14 @@ TakeOrderedAndProject (79) : +- ReusedExchange (48) +- BroadcastExchange (76) +- * HashAggregate (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometColumnarExchange (73) +- * HashAggregate (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) :- * Project (68) : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) + : :- CometNativeColumnarToRow (62) : : +- CometProject (61) : : +- CometFilter (60) : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -95,7 +95,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharC Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] (5) Scan parquet spark_catalog.default.store_sales @@ -117,30 +117,30 @@ Condition : isnotnull(ss_customer_sk#15) Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#15] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] (11) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#20, d_year#21] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] @@ -151,17 +151,17 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#25, year_total#26] Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) @@ -180,7 +180,7 @@ Condition : (isnotnull(c_customer_sk#27) AND isnotnull(static_invoke(CharVarchar Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] Arguments: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40], [c_customer_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#28, 16)) AS c_customer_id#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#29, 20)) AS c_first_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#30, 30)) AS c_last_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#31, 1)) AS c_preferred_cust_flag#38, c_birth_country#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#33, 13)) AS c_login#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#34, 50)) AS c_email_address#40] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [8]: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] (23) Scan parquet spark_catalog.default.store_sales @@ -191,10 +191,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] Condition : isnotnull(ss_customer_sk#41) @@ -202,30 +202,30 @@ Condition : isnotnull(ss_customer_sk#41) Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#27] Right keys [1]: [ss_customer_sk#41] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] Input [12]: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] (29) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#46, d_year#47] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#44] Right keys [1]: [d_date_sk#46] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, d_year#47] Input [12]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, d_year#47] Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))] @@ -236,10 +236,10 @@ Results [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_pr Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] Arguments: hashpartitioning(c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))] @@ -250,7 +250,7 @@ Results [5]: [c_customer_id#35 AS customer_id#50, c_first_name#36 AS customer_fi Input [5]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53, year_total#54] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#50] Join type: Inner @@ -271,7 +271,7 @@ Condition : (isnotnull(c_customer_sk#55) AND isnotnull(static_invoke(CharVarchar Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] Arguments: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68], [c_customer_sk#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#56, 16)) AS c_customer_id#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#57, 20)) AS c_first_name#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#58, 30)) AS c_last_name#65, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#59, 1)) AS c_preferred_cust_flag#66, c_birth_country#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#61, 13)) AS c_login#67, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#62, 50)) AS c_email_address#68] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [8]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68] (42) Scan parquet spark_catalog.default.web_sales @@ -282,10 +282,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Condition : isnotnull(ws_bill_customer_sk#69) @@ -293,30 +293,30 @@ Condition : isnotnull(ws_bill_customer_sk#69) Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#55] Right keys [1]: [ws_bill_customer_sk#69] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] Input [12]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] (48) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#73, d_year#74] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#72] Right keys [1]: [d_date_sk#73] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#74] Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72, d_date_sk#73, d_year#74] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#74] Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] @@ -327,17 +327,17 @@ Results [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cus Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74, sum#76] Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74, sum#76] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74, sum#76] Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#77] Results [2]: [c_customer_id#63 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#77,18,2) AS year_total#79] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#78, year_total#79] Condition : (isnotnull(year_total#79) AND (year_total#79 > 0.00)) @@ -345,13 +345,13 @@ Condition : (isnotnull(year_total#79) AND (year_total#79 > 0.00)) Input [2]: [customer_id#78, year_total#79] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#78] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(58) Project [codegen id : 12] Output [8]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53, year_total#54, year_total#79] Input [9]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53, year_total#54, customer_id#78, year_total#79] @@ -370,7 +370,7 @@ Condition : (isnotnull(c_customer_sk#80) AND isnotnull(static_invoke(CharVarchar Input [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] Arguments: [c_customer_sk#80, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93], [c_customer_sk#80, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#81, 16)) AS c_customer_id#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#82, 20)) AS c_first_name#89, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#83, 30)) AS c_last_name#90, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#84, 1)) AS c_preferred_cust_flag#91, c_birth_country#85, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#86, 13)) AS c_login#92, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#87, 50)) AS c_email_address#93] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [8]: [c_customer_sk#80, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93] (63) Scan parquet spark_catalog.default.web_sales @@ -381,10 +381,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#97), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [4]: [ws_bill_customer_sk#94, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [4]: [ws_bill_customer_sk#94, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] Condition : isnotnull(ws_bill_customer_sk#94) @@ -392,30 +392,30 @@ Condition : isnotnull(ws_bill_customer_sk#94) Input [4]: [ws_bill_customer_sk#94, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#80] Right keys [1]: [ws_bill_customer_sk#94] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] Input [12]: [c_customer_sk#80, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_bill_customer_sk#94, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] (69) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#98, d_year#99] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#97] Right keys [1]: [d_date_sk#98] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_ext_discount_amt#95, ws_ext_list_price#96, d_year#99] Input [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97, d_date_sk#98, d_year#99] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_ext_discount_amt#95, ws_ext_list_price#96, d_year#99] Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#96 - ws_ext_discount_amt#95)))] @@ -426,10 +426,10 @@ Results [9]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cus Input [9]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99, sum#101] Arguments: hashpartitioning(c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [9]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99, sum#101] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [9]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99, sum#101] Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#96 - ws_ext_discount_amt#95)))] @@ -440,13 +440,13 @@ Results [2]: [c_customer_id#88 AS customer_id#102, MakeDecimal(sum(UnscaledValue Input [2]: [customer_id#102, year_total#103] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#102] Join type: Inner Join condition: (CASE WHEN (year_total#79 > 0.00) THEN (year_total#103 / year_total#79) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#54 / year_total#26) ELSE 0E-20 END) -(78) Project [codegen id : 16] +(78) Project [codegen id : 12] Output [4]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53] Input [10]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53, year_total#54, year_total#79, customer_id#102, year_total#103] @@ -458,7 +458,7 @@ Arguments: 100, [customer_id#50 ASC NULLS FIRST, customer_first_name#51 ASC NULL Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometFilter (81) +- CometNativeScan parquet spark_catalog.default.date_dim (80) @@ -474,7 +474,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (83) BroadcastExchange @@ -483,7 +483,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#45 BroadcastExchange (87) -+- * CometColumnarToRow (86) ++- CometNativeColumnarToRow (86) +- CometFilter (85) +- CometNativeScan parquet spark_catalog.default.date_dim (84) @@ -499,7 +499,7 @@ ReadSchema: struct Input [2]: [d_date_sk#46, d_year#47] Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#46)) -(86) CometColumnarToRow [codegen id : 1] +(86) CometNativeColumnarToRow Input [2]: [d_date_sk#46, d_year#47] (87) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt index 0d59f7b833..5d218998be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -23,23 +23,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -49,24 +49,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -76,19 +76,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -98,7 +98,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/simplified.txt index 00a3e659d4..a15087828c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] @@ -7,17 +7,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] @@ -30,69 +30,65 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -102,25 +98,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt index 676d488824..8657242952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#44, d_year#45] Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#44, d_year#45] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt index 157d1d587e..81d240b8dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt index 676d488824..8657242952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt @@ -415,7 +415,7 @@ Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#44, d_year#45] Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#44, d_year#45] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 157d1d587e..81d240b8dd 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 @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/explain.txt index 51f95a9e5b..17456c49e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.web_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) @@ -54,37 +54,37 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10 Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt index 6c2a775097..15648408fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/simplified.txt index c129b42cdb..2b82e74b78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt index affae68344..f8a7566ee4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt index 58ac81ba0d..11c87950e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt index affae68344..f8a7566ee4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 58ac81ba0d..11c87950e6 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/explain.txt index 5b9627c41c..e329a1045d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (87) +- * BroadcastHashJoin Inner BuildRight (86) :- * Filter (68) : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) + : +- CometNativeColumnarToRow (66) : +- CometColumnarExchange (65) : +- * HashAggregate (64) : +- * Project (63) @@ -17,12 +17,12 @@ TakeOrderedAndProject (87) : : : +- BroadcastExchange (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) + : : : :- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (48) : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) + : : : :- CometNativeColumnarToRow (36) : : : : +- CometHashAggregate (35) : : : : +- CometColumnarExchange (34) : : : : +- * HashAggregate (33) @@ -35,7 +35,7 @@ TakeOrderedAndProject (87) : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : +- BroadcastExchange (27) : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) + : : : : : :- CometNativeColumnarToRow (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- BroadcastExchange (25) @@ -47,7 +47,7 @@ TakeOrderedAndProject (87) : : : : : : : +- * ColumnarToRow (14) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometNativeColumnarToRow (18) : : : : : : +- CometFilter (17) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : +- ReusedExchange (22) @@ -64,7 +64,7 @@ TakeOrderedAndProject (87) : : : +- ReusedExchange (43) : : +- BroadcastExchange (58) : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) + : : :- CometNativeColumnarToRow (55) : : : +- CometFilter (54) : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : +- ReusedExchange (56) @@ -72,7 +72,7 @@ TakeOrderedAndProject (87) +- BroadcastExchange (85) +- * Filter (84) +- * HashAggregate (83) - +- * CometColumnarToRow (82) + +- CometNativeColumnarToRow (82) +- CometColumnarExchange (81) +- * HashAggregate (80) +- * Project (79) @@ -96,10 +96,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -114,7 +114,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -143,7 +143,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -172,33 +172,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -303,7 +303,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -313,13 +313,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -327,7 +327,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -344,13 +344,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -416,17 +416,17 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 50] +(70) ColumnarToRow [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(71) Filter [codegen id : 50] +(71) Filter [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) (72) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#59] -(73) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#59] Join type: LeftSemi @@ -435,30 +435,30 @@ Join condition: None (74) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -(75) BroadcastHashJoin [codegen id : 50] +(75) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#60] Join type: Inner Join condition: None -(76) Project [codegen id : 50] +(76) Project [codegen id : 28] Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] (77) ReusedExchange [Reuses operator id: 126] Output [1]: [d_date_sk#64] -(78) BroadcastHashJoin [codegen id : 50] +(78) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#64] Join type: Inner Join condition: None -(79) Project [codegen id : 50] +(79) Project [codegen id : 28] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -(80) HashAggregate [codegen id : 50] +(80) HashAggregate [codegen id : 28] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] @@ -469,17 +469,17 @@ Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(82) CometColumnarToRow [codegen id : 51] +(82) CometNativeColumnarToRow Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -(83) HashAggregate [codegen id : 51] +(83) HashAggregate [codegen id : 29] Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71, count(1)#72] Results [6]: [store AS channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71 AS sales#74, count(1)#72 AS number_sales#75] -(84) Filter [codegen id : 51] +(84) Filter [codegen id : 29] Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -487,7 +487,7 @@ Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(Reu Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 52] +(86) BroadcastHashJoin [codegen id : 30] Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] Join type: Inner @@ -501,7 +501,7 @@ Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * HashAggregate (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometColumnarExchange (105) +- * HashAggregate (104) +- Union (103) @@ -529,19 +529,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#12)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 2] +(89) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] (90) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#79] -(91) BroadcastHashJoin [codegen id : 2] +(91) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#78] Right keys [1]: [d_date_sk#79] Join type: Inner Join condition: None -(92) Project [codegen id : 2] +(92) Project [codegen id : 1] Output [2]: [ss_quantity#76 AS quantity#80, ss_list_price#77 AS list_price#81] Input [4]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#79] @@ -552,19 +552,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#12)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 4] +(94) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] (95) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#85] -(96) BroadcastHashJoin [codegen id : 4] +(96) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#84] Right keys [1]: [d_date_sk#85] Join type: Inner Join condition: None -(97) Project [codegen id : 4] +(97) Project [codegen id : 2] Output [2]: [cs_quantity#82 AS quantity#86, cs_list_price#83 AS list_price#87] Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#85] @@ -575,25 +575,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#12)] ReadSchema: struct -(99) ColumnarToRow [codegen id : 6] +(99) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] (100) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#91] -(101) BroadcastHashJoin [codegen id : 6] +(101) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#90] Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(102) Project [codegen id : 6] +(102) Project [codegen id : 3] Output [2]: [ws_quantity#88 AS quantity#92, ws_list_price#89 AS list_price#93] Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#91] (103) Union -(104) HashAggregate [codegen id : 7] +(104) HashAggregate [codegen id : 4] Input [2]: [quantity#80, list_price#81] Keys: [] Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] @@ -604,10 +604,10 @@ Results [2]: [sum#96, count#97] Input [2]: [sum#96, count#97] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(106) CometColumnarToRow [codegen id : 8] +(106) CometNativeColumnarToRow Input [2]: [sum#96, count#97] -(107) HashAggregate [codegen id : 8] +(107) HashAggregate [codegen id : 5] Input [2]: [sum#96, count#97] Keys: [] Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] @@ -622,7 +622,7 @@ Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (112) -+- * CometColumnarToRow (111) ++- CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometNativeScan parquet spark_catalog.default.date_dim (108) @@ -643,7 +643,7 @@ Condition : ((isnotnull(d_week_seq#100) AND (d_week_seq#100 = ReusedSubquery Sub Input [2]: [d_date_sk#40, d_week_seq#100] Arguments: [d_date_sk#40], [d_date_sk#40] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (112) BroadcastExchange @@ -653,7 +653,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 109 Hosting Expression = ReusedSubquery Subquery scalar-subquery#101, [id=#102] Subquery:7 Hosting operator id = 108 Hosting Expression = Subquery scalar-subquery#101, [id=#102] -* CometColumnarToRow (116) +CometNativeColumnarToRow (116) +- CometProject (115) +- CometFilter (114) +- CometNativeScan parquet spark_catalog.default.date_dim (113) @@ -674,12 +674,12 @@ Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] Arguments: [d_week_seq#103], [d_week_seq#103] -(116) CometColumnarToRow [codegen id : 1] +(116) CometNativeColumnarToRow Input [1]: [d_week_seq#103] Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometNativeScan parquet spark_catalog.default.date_dim (117) @@ -700,7 +700,7 @@ Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1998)) AND (d_year#107 < Input [2]: [d_date_sk#24, d_year#107] Arguments: [d_date_sk#24], [d_date_sk#24] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (121) BroadcastExchange @@ -715,7 +715,7 @@ Subquery:11 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 BroadcastExchange (126) -+- * CometColumnarToRow (125) ++- CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometNativeScan parquet spark_catalog.default.date_dim (122) @@ -736,7 +736,7 @@ Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = ReusedSubquery Sub Input [2]: [d_date_sk#64, d_week_seq#108] Arguments: [d_date_sk#64], [d_date_sk#64] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_date_sk#64] (126) BroadcastExchange @@ -746,7 +746,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:13 Hosting operator id = 123 Hosting Expression = ReusedSubquery Subquery scalar-subquery#109, [id=#110] Subquery:14 Hosting operator id = 122 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* CometColumnarToRow (130) +CometNativeColumnarToRow (130) +- CometProject (129) +- CometFilter (128) +- CometNativeScan parquet spark_catalog.default.date_dim (127) @@ -767,7 +767,7 @@ Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] Arguments: [d_week_seq#111], [d_week_seq#111] -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [1]: [d_week_seq#111] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt index e44d85bdc2..25e8321ad7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject :- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -23,7 +23,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -33,12 +33,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -51,25 +51,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -82,13 +82,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -101,16 +101,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -124,28 +124,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -158,13 +158,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -177,16 +177,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -200,22 +200,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -223,7 +223,7 @@ TakeOrderedAndProject +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -236,25 +236,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -267,13 +267,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -286,16 +286,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -309,28 +309,28 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -343,13 +343,13 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -362,16 +362,16 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -385,22 +385,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim +- Subquery - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/simplified.txt index cc02a716ea..27d519ebbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (30) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #12 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -21,7 +21,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #6 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -56,38 +56,34 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (5) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [brand_id,class_id,category_id] CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -99,23 +95,21 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 - WholeStageCodegen (4) + WholeStageCodegen (2) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #8 - WholeStageCodegen (3) + WholeStageCodegen (1) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -127,18 +121,16 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 - WholeStageCodegen (9) + WholeStageCodegen (4) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -154,10 +146,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 - WholeStageCodegen (23) + WholeStageCodegen (13) BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter @@ -166,14 +158,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (51) + WholeStageCodegen (29) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + WholeStageCodegen (28) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -186,20 +178,16 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/explain.txt index 31cec72dc1..5cf711f3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subqu Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -623,7 +623,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] Subquery:7 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -644,12 +644,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#99] Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -670,7 +670,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -689,7 +689,7 @@ Subquery:13 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:14 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -710,7 +710,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subqu Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -720,7 +720,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:15 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] Subquery:16 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -741,7 +741,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#104] Subquery:17 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt index b32dd95954..49eafd1015 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt @@ -44,13 +44,13 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,7 +72,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -136,7 +136,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -199,13 +199,13 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -227,7 +227,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -291,7 +291,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/simplified.txt index 6b5264b76d..0d84ade8e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/simplified.txt @@ -39,20 +39,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -71,12 +67,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -133,20 +127,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt index 31cec72dc1..5cf711f3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subqu Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -623,7 +623,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] Subquery:7 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -644,12 +644,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#99] Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -670,7 +670,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -689,7 +689,7 @@ Subquery:13 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:14 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -710,7 +710,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subqu Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -720,7 +720,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:15 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] Subquery:16 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -741,7 +741,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#104] Subquery:17 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/extended.txt index b32dd95954..49eafd1015 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/extended.txt @@ -44,13 +44,13 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,7 +72,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -136,7 +136,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -199,13 +199,13 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -227,7 +227,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -291,7 +291,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6b5264b76d..0d84ade8e8 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 @@ -39,20 +39,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -71,12 +67,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -133,20 +127,16 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + ReusedSubquery [d_week_seq] #6 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/explain.txt index d17a5a745a..ebfd767543 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/explain.txt @@ -1,18 +1,18 @@ == Physical Plan == -* CometColumnarToRow (139) +CometNativeColumnarToRow (139) +- CometTakeOrderedAndProject (138) +- CometHashAggregate (137) +- CometColumnarExchange (136) +- * HashAggregate (135) +- Union (134) :- * HashAggregate (105) - : +- * CometColumnarToRow (104) + : +- CometNativeColumnarToRow (104) : +- CometColumnarExchange (103) : +- * HashAggregate (102) : +- Union (101) : :- * Filter (68) : : +- * HashAggregate (67) - : : +- * CometColumnarToRow (66) + : : +- CometNativeColumnarToRow (66) : : +- CometColumnarExchange (65) : : +- * HashAggregate (64) : : +- * Project (63) @@ -26,12 +26,12 @@ : : : : +- BroadcastExchange (51) : : : : +- * Project (50) : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * CometColumnarToRow (6) + : : : : :- CometNativeColumnarToRow (6) : : : : : +- CometFilter (5) : : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : : +- BroadcastExchange (48) : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : :- * CometColumnarToRow (36) + : : : : :- CometNativeColumnarToRow (36) : : : : : +- CometHashAggregate (35) : : : : : +- CometColumnarExchange (34) : : : : : +- * HashAggregate (33) @@ -44,7 +44,7 @@ : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : : +- BroadcastExchange (27) : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * CometColumnarToRow (12) + : : : : : : :- CometNativeColumnarToRow (12) : : : : : : : +- CometFilter (11) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : : +- BroadcastExchange (25) @@ -56,7 +56,7 @@ : : : : : : : : +- * ColumnarToRow (14) : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometNativeColumnarToRow (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : : +- ReusedExchange (22) @@ -73,14 +73,14 @@ : : : : +- ReusedExchange (43) : : : +- BroadcastExchange (58) : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * CometColumnarToRow (55) + : : : :- CometNativeColumnarToRow (55) : : : : +- CometFilter (54) : : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : : +- ReusedExchange (56) : : +- ReusedExchange (61) : :- * Filter (84) : : +- * HashAggregate (83) - : : +- * CometColumnarToRow (82) + : : +- CometNativeColumnarToRow (82) : : +- CometColumnarExchange (81) : : +- * HashAggregate (80) : : +- * Project (79) @@ -96,7 +96,7 @@ : : +- ReusedExchange (77) : +- * Filter (100) : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) + : +- CometNativeColumnarToRow (98) : +- CometColumnarExchange (97) : +- * HashAggregate (96) : +- * Project (95) @@ -111,32 +111,32 @@ : : +- ReusedExchange (90) : +- ReusedExchange (93) :- * HashAggregate (112) - : +- * CometColumnarToRow (111) + : +- CometNativeColumnarToRow (111) : +- CometColumnarExchange (110) : +- * HashAggregate (109) : +- * HashAggregate (108) - : +- * CometColumnarToRow (107) + : +- CometNativeColumnarToRow (107) : +- ReusedExchange (106) :- * HashAggregate (119) - : +- * CometColumnarToRow (118) + : +- CometNativeColumnarToRow (118) : +- CometColumnarExchange (117) : +- * HashAggregate (116) : +- * HashAggregate (115) - : +- * CometColumnarToRow (114) + : +- CometNativeColumnarToRow (114) : +- ReusedExchange (113) :- * HashAggregate (126) - : +- * CometColumnarToRow (125) + : +- CometNativeColumnarToRow (125) : +- CometColumnarExchange (124) : +- * HashAggregate (123) : +- * HashAggregate (122) - : +- * CometColumnarToRow (121) + : +- CometNativeColumnarToRow (121) : +- ReusedExchange (120) +- * HashAggregate (133) - +- * CometColumnarToRow (132) + +- CometNativeColumnarToRow (132) +- CometColumnarExchange (131) +- * HashAggregate (130) +- * HashAggregate (129) - +- * CometColumnarToRow (128) + +- CometNativeColumnarToRow (128) +- ReusedExchange (127) @@ -148,10 +148,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -166,7 +166,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -195,7 +195,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -224,33 +224,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 174] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -355,7 +355,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -365,13 +365,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -379,7 +379,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -396,13 +396,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -468,17 +468,17 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 51] +(70) ColumnarToRow [codegen id : 29] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -(71) Filter [codegen id : 51] +(71) Filter [codegen id : 29] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Condition : isnotnull(cs_item_sk#54) (72) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#58] -(73) BroadcastHashJoin [codegen id : 51] +(73) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#54] Right keys [1]: [ss_item_sk#58] Join type: LeftSemi @@ -487,30 +487,30 @@ Join condition: None (74) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(75) BroadcastHashJoin [codegen id : 51] +(75) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#54] Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(76) Project [codegen id : 51] +(76) Project [codegen id : 29] Output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] (77) ReusedExchange [Reuses operator id: 169] Output [1]: [d_date_sk#63] -(78) BroadcastHashJoin [codegen id : 51] +(78) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_sold_date_sk#57] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(79) Project [codegen id : 51] +(79) Project [codegen id : 29] Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] -(80) HashAggregate [codegen id : 51] +(80) HashAggregate [codegen id : 29] Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] @@ -521,17 +521,17 @@ Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(82) CometColumnarToRow [codegen id : 52] +(82) CometNativeColumnarToRow Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -(83) HashAggregate [codegen id : 52] +(83) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#70, count(1)#71] Results [6]: [catalog AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(84) Filter [codegen id : 52] +(84) Filter [codegen id : 30] Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -543,17 +543,17 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 77] +(86) ColumnarToRow [codegen id : 44] Input [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] -(87) Filter [codegen id : 77] +(87) Filter [codegen id : 44] Input [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] Condition : isnotnull(ws_item_sk#75) (88) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#79] -(89) BroadcastHashJoin [codegen id : 77] +(89) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#75] Right keys [1]: [ss_item_sk#79] Join type: LeftSemi @@ -562,30 +562,30 @@ Join condition: None (90) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(91) BroadcastHashJoin [codegen id : 77] +(91) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#75] Right keys [1]: [i_item_sk#80] Join type: Inner Join condition: None -(92) Project [codegen id : 77] +(92) Project [codegen id : 44] Output [6]: [ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_brand_id#81, i_class_id#82, i_category_id#83] Input [8]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] (93) ReusedExchange [Reuses operator id: 169] Output [1]: [d_date_sk#84] -(94) BroadcastHashJoin [codegen id : 77] +(94) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#78] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(95) Project [codegen id : 77] +(95) Project [codegen id : 44] Output [5]: [ws_quantity#76, ws_list_price#77, i_brand_id#81, i_class_id#82, i_category_id#83] Input [7]: [ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_brand_id#81, i_class_id#82, i_category_id#83, d_date_sk#84] -(96) HashAggregate [codegen id : 77] +(96) HashAggregate [codegen id : 44] Input [5]: [ws_quantity#76, ws_list_price#77, i_brand_id#81, i_class_id#82, i_category_id#83] Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] Functions [2]: [partial_sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77)), partial_count(1)] @@ -596,23 +596,23 @@ Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89 Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(98) CometColumnarToRow [codegen id : 78] +(98) CometNativeColumnarToRow Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] -(99) HashAggregate [codegen id : 78] +(99) HashAggregate [codegen id : 45] Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] Functions [2]: [sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77))#91, count(1)#92] Results [6]: [web AS channel#93, i_brand_id#81, i_class_id#82, i_category_id#83, sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77))#91 AS sales#94, count(1)#92 AS number_sales#95] -(100) Filter [codegen id : 78] +(100) Filter [codegen id : 45] Input [6]: [channel#93, i_brand_id#81, i_class_id#82, i_category_id#83, sales#94, number_sales#95] Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) (101) Union -(102) HashAggregate [codegen id : 79] +(102) HashAggregate [codegen id : 46] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] @@ -623,10 +623,10 @@ Results [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99 Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(104) CometColumnarToRow [codegen id : 80] +(104) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] -(105) HashAggregate [codegen id : 80] +(105) HashAggregate [codegen id : 47] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] @@ -636,17 +636,17 @@ Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(sa (106) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, sum#110, isEmpty#111, sum#112] -(107) CometColumnarToRow [codegen id : 160] +(107) CometNativeColumnarToRow Input [7]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, sum#110, isEmpty#111, sum#112] -(108) HashAggregate [codegen id : 160] +(108) HashAggregate [codegen id : 94] Input [7]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, sum#110, isEmpty#111, sum#112] Keys [4]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109] Functions [2]: [sum(sales#113), sum(number_sales#114)] Aggregate Attributes [2]: [sum(sales#113)#102, sum(number_sales#114)#103] Results [5]: [channel#106, i_brand_id#107, i_class_id#108, sum(sales#113)#102 AS sum_sales#115, sum(number_sales#114)#103 AS number_sales#116] -(109) HashAggregate [codegen id : 160] +(109) HashAggregate [codegen id : 94] Input [5]: [channel#106, i_brand_id#107, i_class_id#108, sum_sales#115, number_sales#116] Keys [3]: [channel#106, i_brand_id#107, i_class_id#108] Functions [2]: [partial_sum(sum_sales#115), partial_sum(number_sales#116)] @@ -657,10 +657,10 @@ Results [6]: [channel#106, i_brand_id#107, i_class_id#108, sum#120, isEmpty#121, Input [6]: [channel#106, i_brand_id#107, i_class_id#108, sum#120, isEmpty#121, sum#122] Arguments: hashpartitioning(channel#106, i_brand_id#107, i_class_id#108, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(111) CometColumnarToRow [codegen id : 161] +(111) CometNativeColumnarToRow Input [6]: [channel#106, i_brand_id#107, i_class_id#108, sum#120, isEmpty#121, sum#122] -(112) HashAggregate [codegen id : 161] +(112) HashAggregate [codegen id : 95] Input [6]: [channel#106, i_brand_id#107, i_class_id#108, sum#120, isEmpty#121, sum#122] Keys [3]: [channel#106, i_brand_id#107, i_class_id#108] Functions [2]: [sum(sum_sales#115), sum(number_sales#116)] @@ -670,17 +670,17 @@ Results [6]: [channel#106, i_brand_id#107, i_class_id#108, null AS i_category_id (113) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#128, i_brand_id#129, i_class_id#130, i_category_id#131, sum#132, isEmpty#133, sum#134] -(114) CometColumnarToRow [codegen id : 241] +(114) CometNativeColumnarToRow Input [7]: [channel#128, i_brand_id#129, i_class_id#130, i_category_id#131, sum#132, isEmpty#133, sum#134] -(115) HashAggregate [codegen id : 241] +(115) HashAggregate [codegen id : 142] Input [7]: [channel#128, i_brand_id#129, i_class_id#130, i_category_id#131, sum#132, isEmpty#133, sum#134] Keys [4]: [channel#128, i_brand_id#129, i_class_id#130, i_category_id#131] Functions [2]: [sum(sales#135), sum(number_sales#136)] Aggregate Attributes [2]: [sum(sales#135)#102, sum(number_sales#136)#103] Results [4]: [channel#128, i_brand_id#129, sum(sales#135)#102 AS sum_sales#137, sum(number_sales#136)#103 AS number_sales#138] -(116) HashAggregate [codegen id : 241] +(116) HashAggregate [codegen id : 142] Input [4]: [channel#128, i_brand_id#129, sum_sales#137, number_sales#138] Keys [2]: [channel#128, i_brand_id#129] Functions [2]: [partial_sum(sum_sales#137), partial_sum(number_sales#138)] @@ -691,10 +691,10 @@ Results [5]: [channel#128, i_brand_id#129, sum#142, isEmpty#143, sum#144] Input [5]: [channel#128, i_brand_id#129, sum#142, isEmpty#143, sum#144] Arguments: hashpartitioning(channel#128, i_brand_id#129, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(118) CometColumnarToRow [codegen id : 242] +(118) CometNativeColumnarToRow Input [5]: [channel#128, i_brand_id#129, sum#142, isEmpty#143, sum#144] -(119) HashAggregate [codegen id : 242] +(119) HashAggregate [codegen id : 143] Input [5]: [channel#128, i_brand_id#129, sum#142, isEmpty#143, sum#144] Keys [2]: [channel#128, i_brand_id#129] Functions [2]: [sum(sum_sales#137), sum(number_sales#138)] @@ -704,17 +704,17 @@ Results [6]: [channel#128, i_brand_id#129, null AS i_class_id#147, null AS i_cat (120) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#151, i_brand_id#152, i_class_id#153, i_category_id#154, sum#155, isEmpty#156, sum#157] -(121) CometColumnarToRow [codegen id : 322] +(121) CometNativeColumnarToRow Input [7]: [channel#151, i_brand_id#152, i_class_id#153, i_category_id#154, sum#155, isEmpty#156, sum#157] -(122) HashAggregate [codegen id : 322] +(122) HashAggregate [codegen id : 190] Input [7]: [channel#151, i_brand_id#152, i_class_id#153, i_category_id#154, sum#155, isEmpty#156, sum#157] Keys [4]: [channel#151, i_brand_id#152, i_class_id#153, i_category_id#154] Functions [2]: [sum(sales#158), sum(number_sales#159)] Aggregate Attributes [2]: [sum(sales#158)#102, sum(number_sales#159)#103] Results [3]: [channel#151, sum(sales#158)#102 AS sum_sales#160, sum(number_sales#159)#103 AS number_sales#161] -(123) HashAggregate [codegen id : 322] +(123) HashAggregate [codegen id : 190] Input [3]: [channel#151, sum_sales#160, number_sales#161] Keys [1]: [channel#151] Functions [2]: [partial_sum(sum_sales#160), partial_sum(number_sales#161)] @@ -725,10 +725,10 @@ Results [4]: [channel#151, sum#165, isEmpty#166, sum#167] Input [4]: [channel#151, sum#165, isEmpty#166, sum#167] Arguments: hashpartitioning(channel#151, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(125) CometColumnarToRow [codegen id : 323] +(125) CometNativeColumnarToRow Input [4]: [channel#151, sum#165, isEmpty#166, sum#167] -(126) HashAggregate [codegen id : 323] +(126) HashAggregate [codegen id : 191] Input [4]: [channel#151, sum#165, isEmpty#166, sum#167] Keys [1]: [channel#151] Functions [2]: [sum(sum_sales#160), sum(number_sales#161)] @@ -738,17 +738,17 @@ Results [6]: [channel#151, null AS i_brand_id#170, null AS i_class_id#171, null (127) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#175, i_brand_id#176, i_class_id#177, i_category_id#178, sum#179, isEmpty#180, sum#181] -(128) CometColumnarToRow [codegen id : 403] +(128) CometNativeColumnarToRow Input [7]: [channel#175, i_brand_id#176, i_class_id#177, i_category_id#178, sum#179, isEmpty#180, sum#181] -(129) HashAggregate [codegen id : 403] +(129) HashAggregate [codegen id : 238] Input [7]: [channel#175, i_brand_id#176, i_class_id#177, i_category_id#178, sum#179, isEmpty#180, sum#181] Keys [4]: [channel#175, i_brand_id#176, i_class_id#177, i_category_id#178] Functions [2]: [sum(sales#182), sum(number_sales#183)] Aggregate Attributes [2]: [sum(sales#182)#102, sum(number_sales#183)#103] Results [2]: [sum(sales#182)#102 AS sum_sales#184, sum(number_sales#183)#103 AS number_sales#185] -(130) HashAggregate [codegen id : 403] +(130) HashAggregate [codegen id : 238] Input [2]: [sum_sales#184, number_sales#185] Keys: [] Functions [2]: [partial_sum(sum_sales#184), partial_sum(number_sales#185)] @@ -759,10 +759,10 @@ Results [3]: [sum#189, isEmpty#190, sum#191] Input [3]: [sum#189, isEmpty#190, sum#191] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(132) CometColumnarToRow [codegen id : 404] +(132) CometNativeColumnarToRow Input [3]: [sum#189, isEmpty#190, sum#191] -(133) HashAggregate [codegen id : 404] +(133) HashAggregate [codegen id : 239] Input [3]: [sum#189, isEmpty#190, sum#191] Keys: [] Functions [2]: [sum(sum_sales#184), sum(number_sales#185)] @@ -771,7 +771,7 @@ Results [6]: [null AS channel#194, null AS i_brand_id#195, null AS i_class_id#19 (134) Union -(135) HashAggregate [codegen id : 405] +(135) HashAggregate [codegen id : 240] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] Functions: [] @@ -791,14 +791,14 @@ Functions: [] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#37 ASC NULLS FIRST,i_class_id#38 ASC NULLS FIRST,i_category_id#39 ASC NULLS FIRST], output=[channel#49,i_brand_id#37,i_class_id#38,i_category_id#39,sum_sales#104,number_sales#105]), [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -(139) CometColumnarToRow [codegen id : 406] +(139) CometNativeColumnarToRow Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] ===== Subqueries ===== Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * HashAggregate (159) -+- * CometColumnarToRow (158) ++- CometNativeColumnarToRow (158) +- CometColumnarExchange (157) +- * HashAggregate (156) +- Union (155) @@ -826,19 +826,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#202), dynamicpruningexpression(ss_sold_date_sk#202 IN dynamicpruning#12)] ReadSchema: struct -(141) ColumnarToRow [codegen id : 2] +(141) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#200, ss_list_price#201, ss_sold_date_sk#202] (142) ReusedExchange [Reuses operator id: 174] Output [1]: [d_date_sk#203] -(143) BroadcastHashJoin [codegen id : 2] +(143) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#202] Right keys [1]: [d_date_sk#203] Join type: Inner Join condition: None -(144) Project [codegen id : 2] +(144) Project [codegen id : 1] Output [2]: [ss_quantity#200 AS quantity#204, ss_list_price#201 AS list_price#205] Input [4]: [ss_quantity#200, ss_list_price#201, ss_sold_date_sk#202, d_date_sk#203] @@ -849,19 +849,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#208), dynamicpruningexpression(cs_sold_date_sk#208 IN dynamicpruning#209)] ReadSchema: struct -(146) ColumnarToRow [codegen id : 4] +(146) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#206, cs_list_price#207, cs_sold_date_sk#208] (147) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#210] -(148) BroadcastHashJoin [codegen id : 4] +(148) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#208] Right keys [1]: [d_date_sk#210] Join type: Inner Join condition: None -(149) Project [codegen id : 4] +(149) Project [codegen id : 2] Output [2]: [cs_quantity#206 AS quantity#211, cs_list_price#207 AS list_price#212] Input [4]: [cs_quantity#206, cs_list_price#207, cs_sold_date_sk#208, d_date_sk#210] @@ -872,25 +872,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#215), dynamicpruningexpression(ws_sold_date_sk#215 IN dynamicpruning#209)] ReadSchema: struct -(151) ColumnarToRow [codegen id : 6] +(151) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#213, ws_list_price#214, ws_sold_date_sk#215] (152) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#216] -(153) BroadcastHashJoin [codegen id : 6] +(153) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#215] Right keys [1]: [d_date_sk#216] Join type: Inner Join condition: None -(154) Project [codegen id : 6] +(154) Project [codegen id : 3] Output [2]: [ws_quantity#213 AS quantity#217, ws_list_price#214 AS list_price#218] Input [4]: [ws_quantity#213, ws_list_price#214, ws_sold_date_sk#215, d_date_sk#216] (155) Union -(156) HashAggregate [codegen id : 7] +(156) HashAggregate [codegen id : 4] Input [2]: [quantity#204, list_price#205] Keys: [] Functions [1]: [partial_avg((cast(quantity#204 as decimal(10,0)) * list_price#205))] @@ -901,10 +901,10 @@ Results [2]: [sum#221, count#222] Input [2]: [sum#221, count#222] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(158) CometColumnarToRow [codegen id : 8] +(158) CometNativeColumnarToRow Input [2]: [sum#221, count#222] -(159) HashAggregate [codegen id : 8] +(159) HashAggregate [codegen id : 5] Input [2]: [sum#221, count#222] Keys: [] Functions [1]: [avg((cast(quantity#204 as decimal(10,0)) * list_price#205))] @@ -915,7 +915,7 @@ Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#202 IN Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#208 IN dynamicpruning#209 BroadcastExchange (164) -+- * CometColumnarToRow (163) ++- CometNativeColumnarToRow (163) +- CometProject (162) +- CometFilter (161) +- CometNativeScan parquet spark_catalog.default.date_dim (160) @@ -936,7 +936,7 @@ Condition : (((isnotnull(d_year#225) AND (d_year#225 >= 1998)) AND (d_year#225 < Input [2]: [d_date_sk#210, d_year#225] Arguments: [d_date_sk#210], [d_date_sk#210] -(163) CometColumnarToRow [codegen id : 1] +(163) CometNativeColumnarToRow Input [1]: [d_date_sk#210] (164) BroadcastExchange @@ -947,7 +947,7 @@ Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#215 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (169) -+- * CometColumnarToRow (168) ++- CometNativeColumnarToRow (168) +- CometProject (167) +- CometFilter (166) +- CometNativeScan parquet spark_catalog.default.date_dim (165) @@ -968,7 +968,7 @@ Condition : ((((isnotnull(d_year#226) AND isnotnull(d_moy#227)) AND (d_year#226 Input [3]: [d_date_sk#40, d_year#226, d_moy#227] Arguments: [d_date_sk#40], [d_date_sk#40] -(168) CometColumnarToRow [codegen id : 1] +(168) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (169) BroadcastExchange @@ -977,7 +977,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (174) -+- * CometColumnarToRow (173) ++- CometNativeColumnarToRow (173) +- CometProject (172) +- CometFilter (171) +- CometNativeScan parquet spark_catalog.default.date_dim (170) @@ -998,7 +998,7 @@ Condition : (((isnotnull(d_year#228) AND (d_year#228 >= 1999)) AND (d_year#228 < Input [2]: [d_date_sk#24, d_year#228] Arguments: [d_date_sk#24], [d_date_sk#24] -(173) CometColumnarToRow [codegen id : 1] +(173) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (174) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt index c6478057d9..0bc07e401e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt @@ -1,18 +1,18 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,12 +32,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -47,12 +47,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -65,19 +65,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -90,13 +90,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -109,16 +109,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -132,28 +132,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -166,13 +166,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -185,16 +185,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -208,23 +208,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -239,12 +239,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -257,13 +257,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -276,16 +276,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -299,28 +299,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -333,13 +333,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -352,16 +352,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -375,23 +375,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -406,12 +406,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -424,13 +424,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -443,16 +443,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -466,28 +466,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -500,13 +500,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -519,16 +519,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -542,32 +542,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -577,7 +577,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -587,12 +587,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -602,12 +602,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -620,19 +620,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -645,13 +645,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -664,16 +664,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -687,28 +687,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -721,13 +721,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -740,16 +740,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -763,23 +763,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -794,12 +794,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -812,13 +812,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -831,16 +831,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -854,28 +854,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -888,13 +888,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -907,16 +907,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -930,23 +930,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -961,12 +961,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -979,13 +979,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -998,16 +998,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1021,28 +1021,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -1055,13 +1055,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -1074,16 +1074,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1097,32 +1097,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -1132,7 +1132,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1142,12 +1142,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1157,12 +1157,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1175,19 +1175,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1200,13 +1200,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1219,16 +1219,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1242,28 +1242,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1276,13 +1276,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1295,16 +1295,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1318,23 +1318,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1349,12 +1349,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1367,13 +1367,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1386,16 +1386,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1409,28 +1409,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1443,13 +1443,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1462,16 +1462,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1485,23 +1485,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -1516,12 +1516,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1534,13 +1534,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1553,16 +1553,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1576,28 +1576,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -1610,13 +1610,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -1629,16 +1629,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1652,32 +1652,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -1687,7 +1687,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1697,12 +1697,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1712,12 +1712,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1730,19 +1730,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1755,13 +1755,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1774,16 +1774,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1797,28 +1797,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1831,13 +1831,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1850,16 +1850,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1873,23 +1873,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1904,12 +1904,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1922,13 +1922,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1941,16 +1941,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1964,28 +1964,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1998,13 +1998,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2017,16 +2017,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2040,23 +2040,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -2071,12 +2071,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -2089,13 +2089,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2108,16 +2108,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2131,28 +2131,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2165,13 +2165,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2184,16 +2184,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2207,32 +2207,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -2242,7 +2242,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2252,12 +2252,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2267,12 +2267,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -2285,19 +2285,19 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -2310,13 +2310,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2329,16 +2329,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2352,28 +2352,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2386,13 +2386,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2405,16 +2405,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2428,23 +2428,23 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -2459,12 +2459,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -2477,13 +2477,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2496,16 +2496,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2519,28 +2519,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2553,13 +2553,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2572,16 +2572,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2595,23 +2595,23 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -2626,12 +2626,12 @@ CometColumnarToRow : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2644,13 +2644,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2663,16 +2663,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2686,28 +2686,28 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -2720,13 +2720,13 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -2739,16 +2739,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2762,16 +2762,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/simplified.txt index 4458723393..69a4e70593 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/simplified.txt @@ -1,276 +1,266 @@ -WholeStageCodegen (406) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + WholeStageCodegen (240) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Union + WholeStageCodegen (47) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (46) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (15) + Filter [sales] + Subquery #3 + WholeStageCodegen (5) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #14 + WholeStageCodegen (4) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (1) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (2) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #15 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (3) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #15 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (14) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + BroadcastExchange #6 + WholeStageCodegen (5) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + CometNativeColumnarToRow + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (3) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (2) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + BroadcastExchange #11 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (4) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #13 + WholeStageCodegen (13) + BroadcastHashJoin [i_item_sk,ss_item_sk] + InputAdapter + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (30) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (29) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (45) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (44) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id] #19 - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #20 - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #21 - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (95) + HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,i_brand_id,i_class_id] #18 + WholeStageCodegen (94) + HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (143) + HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,i_brand_id] #19 + WholeStageCodegen (142) + HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (191) + HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel] #20 + WholeStageCodegen (190) + HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (239) + HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #21 + WholeStageCodegen (238) + HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + 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/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/explain.txt index 22106459a6..6c79bbb481 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/explain.txt @@ -809,7 +809,7 @@ Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#169 IN Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#176 IN dynamicpruning#177 BroadcastExchange (147) -+- * CometColumnarToRow (146) ++- CometNativeColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 < Input [2]: [d_date_sk#178, d_year#179] Arguments: [d_date_sk#178], [d_date_sk#178] -(146) CometColumnarToRow [codegen id : 1] +(146) CometNativeColumnarToRow Input [1]: [d_date_sk#178] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#184 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * CometColumnarToRow (151) ++- CometNativeColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) CometColumnarToRow [codegen id : 1] +(151) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * CometColumnarToRow (156) ++- CometNativeColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#192) AND (d_year#192 >= 1999)) AND (d_year#192 < Input [2]: [d_date_sk#26, d_year#192] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) CometColumnarToRow [codegen id : 1] +(156) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (157) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt index 83deb45432..3bacf3c8bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -219,7 +219,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -283,7 +283,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -360,7 +360,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -424,7 +424,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -497,7 +497,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -525,7 +525,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -547,7 +547,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -611,7 +611,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -688,7 +688,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -752,7 +752,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -829,7 +829,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -893,7 +893,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -966,7 +966,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -994,7 +994,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1016,7 +1016,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1080,7 +1080,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1157,7 +1157,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1221,7 +1221,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1298,7 +1298,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1362,7 +1362,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1435,7 +1435,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1463,7 +1463,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1485,7 +1485,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1549,7 +1549,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1626,7 +1626,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1690,7 +1690,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1767,7 +1767,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1831,7 +1831,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1904,7 +1904,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1932,7 +1932,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1954,7 +1954,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2018,7 +2018,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2095,7 +2095,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2159,7 +2159,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2236,7 +2236,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2300,7 +2300,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/simplified.txt index 758e917335..26ab7f52d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/simplified.txt @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #18 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -56,12 +54,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #5 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -80,12 +76,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt index 22106459a6..6c79bbb481 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt @@ -809,7 +809,7 @@ Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#169 IN Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#176 IN dynamicpruning#177 BroadcastExchange (147) -+- * CometColumnarToRow (146) ++- CometNativeColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 < Input [2]: [d_date_sk#178, d_year#179] Arguments: [d_date_sk#178], [d_date_sk#178] -(146) CometColumnarToRow [codegen id : 1] +(146) CometNativeColumnarToRow Input [1]: [d_date_sk#178] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#184 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * CometColumnarToRow (151) ++- CometNativeColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) CometColumnarToRow [codegen id : 1] +(151) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * CometColumnarToRow (156) ++- CometNativeColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#192) AND (d_year#192 >= 1999)) AND (d_year#192 < Input [2]: [d_date_sk#26, d_year#192] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) CometColumnarToRow [codegen id : 1] +(156) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (157) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/extended.txt index 83deb45432..3bacf3c8bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/extended.txt @@ -28,7 +28,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -219,7 +219,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -283,7 +283,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -360,7 +360,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -424,7 +424,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -497,7 +497,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -525,7 +525,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -547,7 +547,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -611,7 +611,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -688,7 +688,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -752,7 +752,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -829,7 +829,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -893,7 +893,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -966,7 +966,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -994,7 +994,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1016,7 +1016,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1080,7 +1080,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1157,7 +1157,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1221,7 +1221,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1298,7 +1298,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1362,7 +1362,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1435,7 +1435,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1463,7 +1463,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1485,7 +1485,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1549,7 +1549,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1626,7 +1626,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1690,7 +1690,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1767,7 +1767,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1831,7 +1831,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1904,7 +1904,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1932,7 +1932,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1954,7 +1954,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2018,7 +2018,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2095,7 +2095,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2159,7 +2159,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2236,7 +2236,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2300,7 +2300,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 758e917335..26ab7f52d4 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 @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #18 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -56,12 +54,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #5 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -80,12 +76,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/explain.txt index 765cd78ddb..482e21c15b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (161) +- Union (160) :- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -21,32 +21,32 @@ TakeOrderedAndProject (161) : : : : : : : +- * ColumnarToRow (2) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) + : : : : : : +- CometNativeColumnarToRow (7) : : : : : : +- CometProject (6) : : : : : : +- CometFilter (5) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) + : : : : : +- CometNativeColumnarToRow (14) : : : : : +- CometProject (13) : : : : : +- CometFilter (12) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) + : : : : +- CometNativeColumnarToRow (20) : : : : +- CometFilter (19) : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) + : : : +- CometNativeColumnarToRow (27) : : : +- CometProject (26) : : : +- CometFilter (25) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) : : +- ReusedExchange (31) : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometProject (36) : +- CometFilter (35) : +- CometNativeScan parquet spark_catalog.default.item (34) :- * HashAggregate (73) - : +- * CometColumnarToRow (72) + : +- CometNativeColumnarToRow (72) : +- CometColumnarExchange (71) : +- * HashAggregate (70) : +- * Project (69) @@ -68,14 +68,14 @@ TakeOrderedAndProject (161) : : : : : +- ReusedExchange (51) : : : : +- ReusedExchange (54) : : : +- BroadcastExchange (61) - : : : +- * CometColumnarToRow (60) + : : : +- CometNativeColumnarToRow (60) : : : +- CometProject (59) : : : +- CometFilter (58) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (57) : : +- ReusedExchange (64) : +- ReusedExchange (67) :- * HashAggregate (102) - : +- * CometColumnarToRow (101) + : +- CometNativeColumnarToRow (101) : +- CometColumnarExchange (100) : +- * HashAggregate (99) : +- * Project (98) @@ -97,14 +97,14 @@ TakeOrderedAndProject (161) : : : : : +- ReusedExchange (80) : : : : +- ReusedExchange (83) : : : +- BroadcastExchange (90) - : : : +- * CometColumnarToRow (89) + : : : +- CometNativeColumnarToRow (89) : : : +- CometProject (88) : : : +- CometFilter (87) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (86) : : +- ReusedExchange (93) : +- ReusedExchange (96) :- * HashAggregate (131) - : +- * CometColumnarToRow (130) + : +- CometNativeColumnarToRow (130) : +- CometColumnarExchange (129) : +- * HashAggregate (128) : +- * Project (127) @@ -126,14 +126,14 @@ TakeOrderedAndProject (161) : : : : : +- ReusedExchange (109) : : : : +- ReusedExchange (112) : : : +- BroadcastExchange (119) - : : : +- * CometColumnarToRow (118) + : : : +- CometNativeColumnarToRow (118) : : : +- CometProject (117) : : : +- CometFilter (116) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (115) : : +- ReusedExchange (122) : +- ReusedExchange (125) +- * HashAggregate (159) - +- * CometColumnarToRow (158) + +- CometNativeColumnarToRow (158) +- CometColumnarExchange (157) +- * HashAggregate (156) +- * Project (155) @@ -157,7 +157,7 @@ TakeOrderedAndProject (161) : : +- ReusedExchange (144) : +- ReusedExchange (147) +- BroadcastExchange (153) - +- * CometColumnarToRow (152) + +- CometNativeColumnarToRow (152) +- CometFilter (151) +- CometNativeScan parquet spark_catalog.default.item (150) @@ -170,10 +170,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) @@ -192,20 +192,20 @@ Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) A Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [cd_demo_sk#11, cd_dep_count#14] (8) BroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 1] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] @@ -224,20 +224,20 @@ Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#1 Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) CometColumnarToRow [codegen id : 2] +(14) CometNativeColumnarToRow Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] (15) BroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 7] +(16) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(17) Project [codegen id : 7] +(17) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] @@ -252,20 +252,20 @@ ReadSchema: struct Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [1]: [cd_demo_sk#20] (21) BroadcastExchange Input [1]: [cd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 1] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -284,33 +284,33 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) AS ca_state#25, ca_country#24] -(27) CometColumnarToRow [codegen id : 4] +(27) CometNativeColumnarToRow Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (28) BroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(30) Project [codegen id : 7] +(30) Project [codegen id : 1] Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (31) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#26] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] @@ -329,24 +329,24 @@ Condition : isnotnull(i_item_sk#27) Input [2]: [i_item_sk#27, i_item_id#28] Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#28, 16)) AS i_item_id#29] -(37) CometColumnarToRow [codegen id : 6] +(37) CometNativeColumnarToRow Input [2]: [i_item_sk#27, i_item_id#29] (38) BroadcastExchange Input [2]: [i_item_sk#27, i_item_id#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 7] +(39) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(40) Project [codegen id : 7] +(40) Project [codegen id : 1] Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] -(41) HashAggregate [codegen id : 7] +(41) HashAggregate [codegen id : 1] Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] @@ -357,10 +357,10 @@ Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, c Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 8] +(43) CometNativeColumnarToRow Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -(44) HashAggregate [codegen id : 8] +(44) HashAggregate [codegen id : 2] Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] @@ -375,49 +375,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 15] +(46) ColumnarToRow [codegen id : 3] Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -(47) Filter [codegen id : 15] +(47) Filter [codegen id : 3] Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) (48) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#92, cd_dep_count#93] -(49) BroadcastHashJoin [codegen id : 15] +(49) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_cdemo_sk#84] Right keys [1]: [cd_demo_sk#92] Join type: Inner Join condition: None -(50) Project [codegen id : 15] +(50) Project [codegen id : 3] Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93] Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#92, cd_dep_count#93] (51) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] -(52) BroadcastHashJoin [codegen id : 15] +(52) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_customer_sk#83] Right keys [1]: [c_customer_sk#94] Join type: Inner Join condition: None -(53) Project [codegen id : 15] +(53) Project [codegen id : 3] Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] (54) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#98] -(55) BroadcastHashJoin [codegen id : 15] +(55) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#95] Right keys [1]: [cd_demo_sk#98] Join type: Inner Join condition: None -(56) Project [codegen id : 15] +(56) Project [codegen id : 3] Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97] Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97, cd_demo_sk#98] @@ -436,50 +436,50 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#100, Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] Arguments: [ca_address_sk#99, ca_state#102, ca_country#101], [ca_address_sk#99, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#100, 2)) AS ca_state#102, ca_country#101] -(60) CometColumnarToRow [codegen id : 12] +(60) CometNativeColumnarToRow Input [3]: [ca_address_sk#99, ca_state#102, ca_country#101] (61) BroadcastExchange Input [3]: [ca_address_sk#99, ca_state#102, ca_country#101] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(62) BroadcastHashJoin [codegen id : 15] +(62) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#96] Right keys [1]: [ca_address_sk#99] Join type: Inner Join condition: None -(63) Project [codegen id : 15] +(63) Project [codegen id : 3] Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#102, ca_country#101] Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97, ca_address_sk#99, ca_state#102, ca_country#101] (64) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#103] -(65) BroadcastHashJoin [codegen id : 15] +(65) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#91] Right keys [1]: [d_date_sk#103] Join type: Inner Join condition: None -(66) Project [codegen id : 15] +(66) Project [codegen id : 3] Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#102, ca_country#101] Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#102, ca_country#101, d_date_sk#103] (67) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#104, i_item_id#105] -(68) BroadcastHashJoin [codegen id : 15] +(68) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#85] Right keys [1]: [i_item_sk#104] Join type: Inner Join condition: None -(69) Project [codegen id : 15] +(69) Project [codegen id : 3] Output [10]: [i_item_id#105, ca_country#101, ca_state#102, cast(cs_quantity#86 as decimal(12,2)) AS agg1#106, cast(cs_list_price#87 as decimal(12,2)) AS agg2#107, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#108, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#109, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#110, cast(c_birth_year#97 as decimal(12,2)) AS agg6#111, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#112] Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#102, ca_country#101, i_item_sk#104, i_item_id#105] -(70) HashAggregate [codegen id : 15] +(70) HashAggregate [codegen id : 3] Input [10]: [i_item_id#105, ca_country#101, ca_state#102, agg1#106, agg2#107, agg3#108, agg4#109, agg5#110, agg6#111, agg7#112] Keys [3]: [i_item_id#105, ca_country#101, ca_state#102] Functions [7]: [partial_avg(agg1#106), partial_avg(agg2#107), partial_avg(agg3#108), partial_avg(agg4#109), partial_avg(agg5#110), partial_avg(agg6#111), partial_avg(agg7#112)] @@ -490,10 +490,10 @@ Results [17]: [i_item_id#105, ca_country#101, ca_state#102, sum#127, count#128, Input [17]: [i_item_id#105, ca_country#101, ca_state#102, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140] Arguments: hashpartitioning(i_item_id#105, ca_country#101, ca_state#102, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(72) CometColumnarToRow [codegen id : 16] +(72) CometNativeColumnarToRow Input [17]: [i_item_id#105, ca_country#101, ca_state#102, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140] -(73) HashAggregate [codegen id : 16] +(73) HashAggregate [codegen id : 4] Input [17]: [i_item_id#105, ca_country#101, ca_state#102, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140] Keys [3]: [i_item_id#105, ca_country#101, ca_state#102] Functions [7]: [avg(agg1#106), avg(agg2#107), avg(agg3#108), avg(agg4#109), avg(agg5#110), avg(agg6#111), avg(agg7#112)] @@ -508,49 +508,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#164), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 23] +(75) ColumnarToRow [codegen id : 5] Input [9]: [cs_bill_customer_sk#156, cs_bill_cdemo_sk#157, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164] -(76) Filter [codegen id : 23] +(76) Filter [codegen id : 5] Input [9]: [cs_bill_customer_sk#156, cs_bill_cdemo_sk#157, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164] Condition : ((isnotnull(cs_bill_cdemo_sk#157) AND isnotnull(cs_bill_customer_sk#156)) AND isnotnull(cs_item_sk#158)) (77) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#165, cd_dep_count#166] -(78) BroadcastHashJoin [codegen id : 23] +(78) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_bill_cdemo_sk#157] Right keys [1]: [cd_demo_sk#165] Join type: Inner Join condition: None -(79) Project [codegen id : 23] +(79) Project [codegen id : 5] Output [9]: [cs_bill_customer_sk#156, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166] Input [11]: [cs_bill_customer_sk#156, cs_bill_cdemo_sk#157, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_demo_sk#165, cd_dep_count#166] (80) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] -(81) BroadcastHashJoin [codegen id : 23] +(81) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_bill_customer_sk#156] Right keys [1]: [c_customer_sk#167] Join type: Inner Join condition: None -(82) Project [codegen id : 23] +(82) Project [codegen id : 5] Output [11]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] Input [13]: [cs_bill_customer_sk#156, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] (83) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#171] -(84) BroadcastHashJoin [codegen id : 23] +(84) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#168] Right keys [1]: [cd_demo_sk#171] Join type: Inner Join condition: None -(85) Project [codegen id : 23] +(85) Project [codegen id : 5] Output [10]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170] Input [12]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170, cd_demo_sk#171] @@ -569,50 +569,50 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#173, Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] Arguments: [ca_address_sk#172, ca_country#174], [ca_address_sk#172, ca_country#174] -(89) CometColumnarToRow [codegen id : 20] +(89) CometNativeColumnarToRow Input [2]: [ca_address_sk#172, ca_country#174] (90) BroadcastExchange Input [2]: [ca_address_sk#172, ca_country#174] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(91) BroadcastHashJoin [codegen id : 23] +(91) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#169] Right keys [1]: [ca_address_sk#172] Join type: Inner Join condition: None -(92) Project [codegen id : 23] +(92) Project [codegen id : 5] Output [10]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_birth_year#170, ca_country#174] Input [12]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170, ca_address_sk#172, ca_country#174] (93) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#175] -(94) BroadcastHashJoin [codegen id : 23] +(94) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#164] Right keys [1]: [d_date_sk#175] Join type: Inner Join condition: None -(95) Project [codegen id : 23] +(95) Project [codegen id : 5] Output [9]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cd_dep_count#166, c_birth_year#170, ca_country#174] Input [11]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_birth_year#170, ca_country#174, d_date_sk#175] (96) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#176, i_item_id#177] -(97) BroadcastHashJoin [codegen id : 23] +(97) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#158] Right keys [1]: [i_item_sk#176] Join type: Inner Join condition: None -(98) Project [codegen id : 23] +(98) Project [codegen id : 5] Output [9]: [i_item_id#177, ca_country#174, cast(cs_quantity#159 as decimal(12,2)) AS agg1#178, cast(cs_list_price#160 as decimal(12,2)) AS agg2#179, cast(cs_coupon_amt#162 as decimal(12,2)) AS agg3#180, cast(cs_sales_price#161 as decimal(12,2)) AS agg4#181, cast(cs_net_profit#163 as decimal(12,2)) AS agg5#182, cast(c_birth_year#170 as decimal(12,2)) AS agg6#183, cast(cd_dep_count#166 as decimal(12,2)) AS agg7#184] Input [11]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cd_dep_count#166, c_birth_year#170, ca_country#174, i_item_sk#176, i_item_id#177] -(99) HashAggregate [codegen id : 23] +(99) HashAggregate [codegen id : 5] Input [9]: [i_item_id#177, ca_country#174, agg1#178, agg2#179, agg3#180, agg4#181, agg5#182, agg6#183, agg7#184] Keys [2]: [i_item_id#177, ca_country#174] Functions [7]: [partial_avg(agg1#178), partial_avg(agg2#179), partial_avg(agg3#180), partial_avg(agg4#181), partial_avg(agg5#182), partial_avg(agg6#183), partial_avg(agg7#184)] @@ -623,10 +623,10 @@ Results [16]: [i_item_id#177, ca_country#174, sum#199, count#200, sum#201, count Input [16]: [i_item_id#177, ca_country#174, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212] Arguments: hashpartitioning(i_item_id#177, ca_country#174, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(101) CometColumnarToRow [codegen id : 24] +(101) CometNativeColumnarToRow Input [16]: [i_item_id#177, ca_country#174, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212] -(102) HashAggregate [codegen id : 24] +(102) HashAggregate [codegen id : 6] Input [16]: [i_item_id#177, ca_country#174, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212] Keys [2]: [i_item_id#177, ca_country#174] Functions [7]: [avg(agg1#178), avg(agg2#179), avg(agg3#180), avg(agg4#181), avg(agg5#182), avg(agg6#183), avg(agg7#184)] @@ -641,49 +641,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#237), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 31] +(104) ColumnarToRow [codegen id : 7] Input [9]: [cs_bill_customer_sk#229, cs_bill_cdemo_sk#230, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237] -(105) Filter [codegen id : 31] +(105) Filter [codegen id : 7] Input [9]: [cs_bill_customer_sk#229, cs_bill_cdemo_sk#230, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237] Condition : ((isnotnull(cs_bill_cdemo_sk#230) AND isnotnull(cs_bill_customer_sk#229)) AND isnotnull(cs_item_sk#231)) (106) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#238, cd_dep_count#239] -(107) BroadcastHashJoin [codegen id : 31] +(107) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_bill_cdemo_sk#230] Right keys [1]: [cd_demo_sk#238] Join type: Inner Join condition: None -(108) Project [codegen id : 31] +(108) Project [codegen id : 7] Output [9]: [cs_bill_customer_sk#229, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239] Input [11]: [cs_bill_customer_sk#229, cs_bill_cdemo_sk#230, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_demo_sk#238, cd_dep_count#239] (109) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] -(110) BroadcastHashJoin [codegen id : 31] +(110) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_bill_customer_sk#229] Right keys [1]: [c_customer_sk#240] Join type: Inner Join condition: None -(111) Project [codegen id : 31] +(111) Project [codegen id : 7] Output [11]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] Input [13]: [cs_bill_customer_sk#229, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] (112) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#244] -(113) BroadcastHashJoin [codegen id : 31] +(113) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_current_cdemo_sk#241] Right keys [1]: [cd_demo_sk#244] Join type: Inner Join condition: None -(114) Project [codegen id : 31] +(114) Project [codegen id : 7] Output [10]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243] Input [12]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243, cd_demo_sk#244] @@ -702,50 +702,50 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#246, Input [2]: [ca_address_sk#245, ca_state#246] Arguments: [ca_address_sk#245], [ca_address_sk#245] -(118) CometColumnarToRow [codegen id : 28] +(118) CometNativeColumnarToRow Input [1]: [ca_address_sk#245] (119) BroadcastExchange Input [1]: [ca_address_sk#245] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(120) BroadcastHashJoin [codegen id : 31] +(120) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_current_addr_sk#242] Right keys [1]: [ca_address_sk#245] Join type: Inner Join condition: None -(121) Project [codegen id : 31] +(121) Project [codegen id : 7] Output [9]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_birth_year#243] Input [11]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243, ca_address_sk#245] (122) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#247] -(123) BroadcastHashJoin [codegen id : 31] +(123) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#237] Right keys [1]: [d_date_sk#247] Join type: Inner Join condition: None -(124) Project [codegen id : 31] +(124) Project [codegen id : 7] Output [8]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cd_dep_count#239, c_birth_year#243] Input [10]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_birth_year#243, d_date_sk#247] (125) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#248, i_item_id#249] -(126) BroadcastHashJoin [codegen id : 31] +(126) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_item_sk#231] Right keys [1]: [i_item_sk#248] Join type: Inner Join condition: None -(127) Project [codegen id : 31] +(127) Project [codegen id : 7] Output [8]: [i_item_id#249, cast(cs_quantity#232 as decimal(12,2)) AS agg1#250, cast(cs_list_price#233 as decimal(12,2)) AS agg2#251, cast(cs_coupon_amt#235 as decimal(12,2)) AS agg3#252, cast(cs_sales_price#234 as decimal(12,2)) AS agg4#253, cast(cs_net_profit#236 as decimal(12,2)) AS agg5#254, cast(c_birth_year#243 as decimal(12,2)) AS agg6#255, cast(cd_dep_count#239 as decimal(12,2)) AS agg7#256] Input [10]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cd_dep_count#239, c_birth_year#243, i_item_sk#248, i_item_id#249] -(128) HashAggregate [codegen id : 31] +(128) HashAggregate [codegen id : 7] Input [8]: [i_item_id#249, agg1#250, agg2#251, agg3#252, agg4#253, agg5#254, agg6#255, agg7#256] Keys [1]: [i_item_id#249] Functions [7]: [partial_avg(agg1#250), partial_avg(agg2#251), partial_avg(agg3#252), partial_avg(agg4#253), partial_avg(agg5#254), partial_avg(agg6#255), partial_avg(agg7#256)] @@ -756,10 +756,10 @@ Results [15]: [i_item_id#249, sum#271, count#272, sum#273, count#274, sum#275, c Input [15]: [i_item_id#249, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280, sum#281, count#282, sum#283, count#284] Arguments: hashpartitioning(i_item_id#249, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(130) CometColumnarToRow [codegen id : 32] +(130) CometNativeColumnarToRow Input [15]: [i_item_id#249, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280, sum#281, count#282, sum#283, count#284] -(131) HashAggregate [codegen id : 32] +(131) HashAggregate [codegen id : 8] Input [15]: [i_item_id#249, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280, sum#281, count#282, sum#283, count#284] Keys [1]: [i_item_id#249] Functions [7]: [avg(agg1#250), avg(agg2#251), avg(agg3#252), avg(agg4#253), avg(agg5#254), avg(agg6#255), avg(agg7#256)] @@ -774,75 +774,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#310), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 39] +(133) ColumnarToRow [codegen id : 9] Input [9]: [cs_bill_customer_sk#302, cs_bill_cdemo_sk#303, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310] -(134) Filter [codegen id : 39] +(134) Filter [codegen id : 9] Input [9]: [cs_bill_customer_sk#302, cs_bill_cdemo_sk#303, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310] Condition : ((isnotnull(cs_bill_cdemo_sk#303) AND isnotnull(cs_bill_customer_sk#302)) AND isnotnull(cs_item_sk#304)) (135) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#311, cd_dep_count#312] -(136) BroadcastHashJoin [codegen id : 39] +(136) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_bill_cdemo_sk#303] Right keys [1]: [cd_demo_sk#311] Join type: Inner Join condition: None -(137) Project [codegen id : 39] +(137) Project [codegen id : 9] Output [9]: [cs_bill_customer_sk#302, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312] Input [11]: [cs_bill_customer_sk#302, cs_bill_cdemo_sk#303, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_demo_sk#311, cd_dep_count#312] (138) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] -(139) BroadcastHashJoin [codegen id : 39] +(139) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_bill_customer_sk#302] Right keys [1]: [c_customer_sk#313] Join type: Inner Join condition: None -(140) Project [codegen id : 39] +(140) Project [codegen id : 9] Output [11]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] Input [13]: [cs_bill_customer_sk#302, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] (141) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#317] -(142) BroadcastHashJoin [codegen id : 39] +(142) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#314] Right keys [1]: [cd_demo_sk#317] Join type: Inner Join condition: None -(143) Project [codegen id : 39] +(143) Project [codegen id : 9] Output [10]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316] Input [12]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316, cd_demo_sk#317] (144) ReusedExchange [Reuses operator id: 119] Output [1]: [ca_address_sk#318] -(145) BroadcastHashJoin [codegen id : 39] +(145) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#315] Right keys [1]: [ca_address_sk#318] Join type: Inner Join condition: None -(146) Project [codegen id : 39] +(146) Project [codegen id : 9] Output [9]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_birth_year#316] Input [11]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316, ca_address_sk#318] (147) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#319] -(148) BroadcastHashJoin [codegen id : 39] +(148) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#310] Right keys [1]: [d_date_sk#319] Join type: Inner Join condition: None -(149) Project [codegen id : 39] +(149) Project [codegen id : 9] Output [8]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cd_dep_count#312, c_birth_year#316] Input [10]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_birth_year#316, d_date_sk#319] @@ -857,24 +857,24 @@ ReadSchema: struct Input [1]: [i_item_sk#320] Condition : isnotnull(i_item_sk#320) -(152) CometColumnarToRow [codegen id : 38] +(152) CometNativeColumnarToRow Input [1]: [i_item_sk#320] (153) BroadcastExchange Input [1]: [i_item_sk#320] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(154) BroadcastHashJoin [codegen id : 39] +(154) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_item_sk#304] Right keys [1]: [i_item_sk#320] Join type: Inner Join condition: None -(155) Project [codegen id : 39] +(155) Project [codegen id : 9] Output [7]: [cast(cs_quantity#305 as decimal(12,2)) AS agg1#321, cast(cs_list_price#306 as decimal(12,2)) AS agg2#322, cast(cs_coupon_amt#308 as decimal(12,2)) AS agg3#323, cast(cs_sales_price#307 as decimal(12,2)) AS agg4#324, cast(cs_net_profit#309 as decimal(12,2)) AS agg5#325, cast(c_birth_year#316 as decimal(12,2)) AS agg6#326, cast(cd_dep_count#312 as decimal(12,2)) AS agg7#327] Input [9]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cd_dep_count#312, c_birth_year#316, i_item_sk#320] -(156) HashAggregate [codegen id : 39] +(156) HashAggregate [codegen id : 9] Input [7]: [agg1#321, agg2#322, agg3#323, agg4#324, agg5#325, agg6#326, agg7#327] Keys: [] Functions [7]: [partial_avg(agg1#321), partial_avg(agg2#322), partial_avg(agg3#323), partial_avg(agg4#324), partial_avg(agg5#325), partial_avg(agg6#326), partial_avg(agg7#327)] @@ -885,10 +885,10 @@ Results [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#3 Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(158) CometColumnarToRow [codegen id : 40] +(158) CometNativeColumnarToRow Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] -(159) HashAggregate [codegen id : 40] +(159) HashAggregate [codegen id : 10] Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] Keys: [] Functions [7]: [avg(agg1#321), avg(agg2#322), avg(agg3#323), avg(agg4#324), avg(agg5#325), avg(agg6#326), avg(agg7#327)] @@ -905,7 +905,7 @@ Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (166) -+- * CometColumnarToRow (165) ++- CometNativeColumnarToRow (165) +- CometProject (164) +- CometFilter (163) +- CometNativeScan parquet spark_catalog.default.date_dim (162) @@ -926,7 +926,7 @@ Condition : ((isnotnull(d_year#374) AND (d_year#374 = 2001)) AND isnotnull(d_dat Input [2]: [d_date_sk#26, d_year#374] Arguments: [d_date_sk#26], [d_date_sk#26] -(165) CometColumnarToRow [codegen id : 1] +(165) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (166) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt index eb411c16e8..8ad4e890ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,41 +21,41 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -75,36 +75,36 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -124,36 +124,36 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -173,36 +173,36 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -222,31 +222,31 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/simplified.txt index 71745258b5..a123d3d8ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] Union - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -25,59 +25,47 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeColumnarToRow + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeColumnarToRow + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometNativeColumnarToRow + CometFilter [cd_demo_sk] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) + WholeStageCodegen (3) HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -104,22 +92,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #9 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) + WholeStageCodegen (6) HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) + WholeStageCodegen (5) HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -146,22 +132,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #11 - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #12 - WholeStageCodegen (31) + WholeStageCodegen (7) HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -188,22 +172,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #13 - WholeStageCodegen (28) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) + WholeStageCodegen (10) HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #14 - WholeStageCodegen (39) + WholeStageCodegen (9) HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -234,8 +216,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #15 - WholeStageCodegen (38) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/explain.txt index 40d7f1ecde..01679e5811 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/explain.txt @@ -807,7 +807,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, a Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (151) -+- * CometColumnarToRow (150) ++- CometNativeColumnarToRow (150) +- CometProject (149) +- CometFilter (148) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) @@ -828,7 +828,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(150) CometColumnarToRow [codegen id : 1] +(150) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (151) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt index b18a444bb9..148f6d4a6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/simplified.txt index 1572a2a240..ce665c5294 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt index 40d7f1ecde..01679e5811 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt @@ -807,7 +807,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, a Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (151) -+- * CometColumnarToRow (150) ++- CometNativeColumnarToRow (150) +- CometProject (149) +- CometFilter (148) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) @@ -828,7 +828,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(150) CometColumnarToRow [codegen id : 1] +(150) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (151) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/extended.txt index b18a444bb9..148f6d4a6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 1572a2a240..ce665c5294 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 @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/explain.txt index 4da3e0a758..9a15a77dd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.catalog_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) @@ -54,37 +54,37 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10 Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt index 369ec68bb4..6634edcbed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/simplified.txt index 7bc0779e53..0c80fb3016 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt index 2e7abc07bb..84bfcfa536 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt index cf18e68a3d..7d1b101ff1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt index 2e7abc07bb..84bfcfa536 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 cf18e68a3d..7d1b101ff1 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q22.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/explain.txt index 0a6d891afc..cdffa5df09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Expand (19) @@ -16,12 +16,12 @@ TakeOrderedAndProject (24) : : : +- Scan parquet spark_catalog.default.inventory (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) +- BroadcastExchange (16) - +- * CometColumnarToRow (15) + +- CometNativeColumnarToRow (15) +- CometNativeScan parquet spark_catalog.default.warehouse (14) @@ -33,23 +33,23 @@ PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] Condition : isnotnull(inv_item_sk#1) (4) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] @@ -68,20 +68,20 @@ Condition : isnotnull(i_item_sk#6) Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#7, 50)) AS i_brand#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#8, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#9, 50)) AS i_category#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#10, 50)) AS i_product_name#14] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] (11) BroadcastExchange Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] @@ -91,26 +91,26 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(15) CometColumnarToRow [codegen id : 3] +(15) CometNativeColumnarToRow Input: [] (16) BroadcastExchange Input: [] Arguments: IdentityBroadcastMode, [plan_id=2] -(17) BroadcastNestedLoopJoin [codegen id : 4] +(17) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -(19) Expand [codegen id : 4] +(19) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] @@ -121,10 +121,10 @@ Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_gr Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [avg(inv_quantity_on_hand#2)] @@ -139,7 +139,7 @@ Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -160,7 +160,7 @@ Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#26] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt index fb0e21025e..108e3c13f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -15,22 +15,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/simplified.txt index 457dff46e2..954f5cb42e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,25 +19,19 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometNativeScan parquet spark_catalog.default.warehouse + CometNativeColumnarToRow + CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/explain.txt index 1f6f727512..e40cb44831 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/explain.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject (26) : +- CometFilter (10) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) @@ -95,7 +95,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(15) CometColumnarToRow [codegen id : 2] +(15) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse @@ -104,26 +104,26 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(17) CometColumnarToRow [codegen id : 1] +(17) CometNativeColumnarToRow Input: [] (18) BroadcastExchange Input: [] Arguments: IdentityBroadcastMode, [plan_id=1] -(19) BroadcastNestedLoopJoin [codegen id : 2] +(19) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(20) Project [codegen id : 2] +(20) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(21) Expand [codegen id : 2] +(21) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] @@ -134,10 +134,10 @@ Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_gr Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(24) CometColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(25) HashAggregate [codegen id : 3] +(25) HashAggregate [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [avg(inv_quantity_on_hand#2)] @@ -152,7 +152,7 @@ Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt index 6cb89465d2..5da4848ebe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -28,7 +28,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/simplified.txt index 221c6063ce..11d581a487 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,12 +19,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -35,7 +33,5 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + CometNativeColumnarToRow + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt index 1f6f727512..e40cb44831 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject (26) : +- CometFilter (10) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) @@ -95,7 +95,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(15) CometColumnarToRow [codegen id : 2] +(15) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse @@ -104,26 +104,26 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(17) CometColumnarToRow [codegen id : 1] +(17) CometNativeColumnarToRow Input: [] (18) BroadcastExchange Input: [] Arguments: IdentityBroadcastMode, [plan_id=1] -(19) BroadcastNestedLoopJoin [codegen id : 2] +(19) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(20) Project [codegen id : 2] +(20) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(21) Expand [codegen id : 2] +(21) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] @@ -134,10 +134,10 @@ Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_gr Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(24) CometColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(25) HashAggregate [codegen id : 3] +(25) HashAggregate [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [avg(inv_quantity_on_hand#2)] @@ -152,7 +152,7 @@ Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt index 6cb89465d2..5da4848ebe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -28,7 +28,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt index 221c6063ce..11d581a487 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,12 +19,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -35,7 +33,5 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + CometNativeColumnarToRow + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/explain.txt index 9842213949..dbf45144df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/explain.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject (55) :- * HashAggregate (25) : +- * HashAggregate (24) : +- * HashAggregate (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometColumnarExchange (21) : +- * HashAggregate (20) : +- * Project (19) @@ -18,41 +18,41 @@ TakeOrderedAndProject (55) : : : : +- Scan parquet spark_catalog.default.inventory (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.item (7) : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.warehouse (14) :- * HashAggregate (32) - : +- * CometColumnarToRow (31) + : +- CometNativeColumnarToRow (31) : +- CometColumnarExchange (30) : +- * HashAggregate (29) : +- * HashAggregate (28) - : +- * CometColumnarToRow (27) + : +- CometNativeColumnarToRow (27) : +- ReusedExchange (26) :- * HashAggregate (39) - : +- * CometColumnarToRow (38) + : +- CometNativeColumnarToRow (38) : +- CometColumnarExchange (37) : +- * HashAggregate (36) : +- * HashAggregate (35) - : +- * CometColumnarToRow (34) + : +- CometNativeColumnarToRow (34) : +- ReusedExchange (33) :- * HashAggregate (46) - : +- * CometColumnarToRow (45) + : +- CometNativeColumnarToRow (45) : +- CometColumnarExchange (44) : +- * HashAggregate (43) : +- * HashAggregate (42) - : +- * CometColumnarToRow (41) + : +- CometNativeColumnarToRow (41) : +- ReusedExchange (40) +- * HashAggregate (53) - +- * CometColumnarToRow (52) + +- CometNativeColumnarToRow (52) +- CometColumnarExchange (51) +- * HashAggregate (50) +- * HashAggregate (49) - +- * CometColumnarToRow (48) + +- CometNativeColumnarToRow (48) +- ReusedExchange (47) @@ -64,23 +64,23 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (4) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] @@ -99,20 +99,20 @@ Condition : isnotnull(i_item_sk#7) Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#8, 50)) AS i_brand#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#11, 50)) AS i_product_name#15] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] (11) BroadcastExchange Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] @@ -127,24 +127,24 @@ ReadSchema: struct Input [1]: [w_warehouse_sk#16] Condition : isnotnull(w_warehouse_sk#16) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [w_warehouse_sk#16] (17) BroadcastExchange Input [1]: [w_warehouse_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#16] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] @@ -155,24 +155,24 @@ Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#3)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] Results [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, avg(inv_quantity_on_hand#3)#21 AS qoh#22] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, qoh#22] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [partial_avg(qoh#22)] Aggregate Attributes [2]: [sum#23, count#24] Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#25, count#26] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 2] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#25, count#26] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(qoh#22)] @@ -182,17 +182,17 @@ Results [5]: [i_product_name#15 AS i_product_name#28, i_brand#12 AS i_brand#29, (26) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] -(27) CometColumnarToRow [codegen id : 10] +(27) CometNativeColumnarToRow Input [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] -(28) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 4] Input [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] Keys [4]: [i_product_name#33, i_brand#34, i_class#35, i_category#36] Functions [1]: [avg(inv_quantity_on_hand#39)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#39)#21] Results [4]: [i_product_name#33, i_brand#34, i_class#35, avg(inv_quantity_on_hand#39)#21 AS qoh#40] -(29) HashAggregate [codegen id : 10] +(29) HashAggregate [codegen id : 4] Input [4]: [i_product_name#33, i_brand#34, i_class#35, qoh#40] Keys [3]: [i_product_name#33, i_brand#34, i_class#35] Functions [1]: [partial_avg(qoh#40)] @@ -203,10 +203,10 @@ Results [5]: [i_product_name#33, i_brand#34, i_class#35, sum#43, count#44] Input [5]: [i_product_name#33, i_brand#34, i_class#35, sum#43, count#44] Arguments: hashpartitioning(i_product_name#33, i_brand#34, i_class#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(31) CometColumnarToRow [codegen id : 11] +(31) CometNativeColumnarToRow Input [5]: [i_product_name#33, i_brand#34, i_class#35, sum#43, count#44] -(32) HashAggregate [codegen id : 11] +(32) HashAggregate [codegen id : 5] Input [5]: [i_product_name#33, i_brand#34, i_class#35, sum#43, count#44] Keys [3]: [i_product_name#33, i_brand#34, i_class#35] Functions [1]: [avg(qoh#40)] @@ -216,17 +216,17 @@ Results [5]: [i_product_name#33, i_brand#34, i_class#35, null AS i_category#46, (33) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#48, i_brand#49, i_class#50, i_category#51, sum#52, count#53] -(34) CometColumnarToRow [codegen id : 16] +(34) CometNativeColumnarToRow Input [6]: [i_product_name#48, i_brand#49, i_class#50, i_category#51, sum#52, count#53] -(35) HashAggregate [codegen id : 16] +(35) HashAggregate [codegen id : 7] Input [6]: [i_product_name#48, i_brand#49, i_class#50, i_category#51, sum#52, count#53] Keys [4]: [i_product_name#48, i_brand#49, i_class#50, i_category#51] Functions [1]: [avg(inv_quantity_on_hand#54)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#54)#21] Results [3]: [i_product_name#48, i_brand#49, avg(inv_quantity_on_hand#54)#21 AS qoh#55] -(36) HashAggregate [codegen id : 16] +(36) HashAggregate [codegen id : 7] Input [3]: [i_product_name#48, i_brand#49, qoh#55] Keys [2]: [i_product_name#48, i_brand#49] Functions [1]: [partial_avg(qoh#55)] @@ -237,10 +237,10 @@ Results [4]: [i_product_name#48, i_brand#49, sum#58, count#59] Input [4]: [i_product_name#48, i_brand#49, sum#58, count#59] Arguments: hashpartitioning(i_product_name#48, i_brand#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(38) CometColumnarToRow [codegen id : 17] +(38) CometNativeColumnarToRow Input [4]: [i_product_name#48, i_brand#49, sum#58, count#59] -(39) HashAggregate [codegen id : 17] +(39) HashAggregate [codegen id : 8] Input [4]: [i_product_name#48, i_brand#49, sum#58, count#59] Keys [2]: [i_product_name#48, i_brand#49] Functions [1]: [avg(qoh#55)] @@ -250,17 +250,17 @@ Results [5]: [i_product_name#48, i_brand#49, null AS i_class#61, null AS i_categ (40) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] -(41) CometColumnarToRow [codegen id : 22] +(41) CometNativeColumnarToRow Input [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] -(42) HashAggregate [codegen id : 22] +(42) HashAggregate [codegen id : 10] Input [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] Keys [4]: [i_product_name#64, i_brand#65, i_class#66, i_category#67] Functions [1]: [avg(inv_quantity_on_hand#70)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#70)#21] Results [2]: [i_product_name#64, avg(inv_quantity_on_hand#70)#21 AS qoh#71] -(43) HashAggregate [codegen id : 22] +(43) HashAggregate [codegen id : 10] Input [2]: [i_product_name#64, qoh#71] Keys [1]: [i_product_name#64] Functions [1]: [partial_avg(qoh#71)] @@ -271,10 +271,10 @@ Results [3]: [i_product_name#64, sum#74, count#75] Input [3]: [i_product_name#64, sum#74, count#75] Arguments: hashpartitioning(i_product_name#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(45) CometColumnarToRow [codegen id : 23] +(45) CometNativeColumnarToRow Input [3]: [i_product_name#64, sum#74, count#75] -(46) HashAggregate [codegen id : 23] +(46) HashAggregate [codegen id : 11] Input [3]: [i_product_name#64, sum#74, count#75] Keys [1]: [i_product_name#64] Functions [1]: [avg(qoh#71)] @@ -284,17 +284,17 @@ Results [5]: [i_product_name#64, null AS i_brand#77, null AS i_class#78, null AS (47) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] -(48) CometColumnarToRow [codegen id : 28] +(48) CometNativeColumnarToRow Input [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] -(49) HashAggregate [codegen id : 28] +(49) HashAggregate [codegen id : 13] Input [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] Keys [4]: [i_product_name#81, i_brand#82, i_class#83, i_category#84] Functions [1]: [avg(inv_quantity_on_hand#87)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#87)#21] Results [1]: [avg(inv_quantity_on_hand#87)#21 AS qoh#88] -(50) HashAggregate [codegen id : 28] +(50) HashAggregate [codegen id : 13] Input [1]: [qoh#88] Keys: [] Functions [1]: [partial_avg(qoh#88)] @@ -305,10 +305,10 @@ Results [2]: [sum#91, count#92] Input [2]: [sum#91, count#92] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(52) CometColumnarToRow [codegen id : 29] +(52) CometNativeColumnarToRow Input [2]: [sum#91, count#92] -(53) HashAggregate [codegen id : 29] +(53) HashAggregate [codegen id : 14] Input [2]: [sum#91, count#92] Keys: [] Functions [1]: [avg(qoh#88)] @@ -325,7 +325,7 @@ Arguments: 100, [qoh#32 ASC NULLS FIRST, i_product_name#28 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (60) -+- * CometColumnarToRow (59) ++- CometNativeColumnarToRow (59) +- CometProject (58) +- CometFilter (57) +- CometNativeScan parquet spark_catalog.default.date_dim (56) @@ -346,7 +346,7 @@ Condition : (((isnotnull(d_month_seq#99) AND (d_month_seq#99 >= 1212)) AND (d_mo Input [2]: [d_date_sk#6, d_month_seq#99] Arguments: [d_date_sk#6], [d_date_sk#6] -(59) CometColumnarToRow [codegen id : 1] +(59) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (60) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt index 543f666a74..eaa5438d91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject :- HashAggregate : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -17,30 +17,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -54,30 +54,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -91,30 +91,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -128,30 +128,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -165,22 +165,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/simplified.txt index 239424b0a0..2e7bb5081d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/simplified.txt @@ -1,13 +1,13 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] @@ -21,70 +21,64 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) + CometNativeColumnarToRow + CometFilter [w_warehouse_sk] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + WholeStageCodegen (5) HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) + WholeStageCodegen (4) HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) + WholeStageCodegen (11) HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name] #7 - WholeStageCodegen (22) + WholeStageCodegen (10) HashAggregate [i_product_name,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) + WholeStageCodegen (14) HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #8 - WholeStageCodegen (28) + WholeStageCodegen (13) HashAggregate [qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow 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/q22a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/explain.txt index b1912021ee..163ca06d72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/explain.txt @@ -270,7 +270,7 @@ Input [5]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt index c018673888..4ccef655b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/simplified.txt index d68adf3747..0183ee81d1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt index b1912021ee..163ca06d72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt @@ -270,7 +270,7 @@ Input [5]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/extended.txt index c018673888..4ccef655b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d68adf3747..0183ee81d1 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/explain.txt index 6ca8ff436b..998cd4d7f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/explain.txt @@ -1,19 +1,19 @@ == Physical Plan == -* CometColumnarToRow (50) +CometNativeColumnarToRow (50) +- CometSort (49) +- CometColumnarExchange (48) +- * Filter (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) + :- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometProject (24) @@ -45,7 +45,7 @@ : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_address (32) @@ -190,7 +190,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 2] +(31) CometNativeColumnarToRow Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] (32) CometNativeScan parquet spark_catalog.default.customer_address @@ -208,24 +208,24 @@ Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isno Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#34, 2)) AS ca_state#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#35, 10)) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] (36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -236,17 +236,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometNativeColumnarToRow Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] @@ -257,17 +257,17 @@ Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty# Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometNativeColumnarToRow Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) @@ -279,23 +279,23 @@ Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(50) CometNativeColumnarToRow Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] * HashAggregate (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometColumnarExchange (78) +- * HashAggregate (77) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- CometColumnarExchange (74) +- * HashAggregate (73) +- * Project (72) +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) + :- CometNativeColumnarToRow (69) : +- CometProject (68) : +- CometBroadcastHashJoin (67) : :- CometProject (65) @@ -392,23 +392,23 @@ Arguments: [ss_customer_sk#52], [c_customer_sk#71], Inner, BuildRight Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] Arguments: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75], [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -(69) CometColumnarToRow [codegen id : 2] +(69) CometNativeColumnarToRow Input [13]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] (70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] -(71) BroadcastHashJoin [codegen id : 2] +(71) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#72, c_birth_country#75, s_zip#61] Right keys [3]: [ca_address_sk#76, upper(ca_country#79), ca_zip#78] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(72) Project [codegen id : 1] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75, ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] -(73) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] @@ -419,17 +419,17 @@ Results [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_ Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] Arguments: hashpartitioning(c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometColumnarToRow [codegen id : 3] +(75) CometNativeColumnarToRow Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] -(76) HashAggregate [codegen id : 3] +(76) HashAggregate [codegen id : 2] Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#82] -(77) HashAggregate [codegen id : 3] +(77) HashAggregate [codegen id : 2] Input [1]: [netpaid#82] Keys: [] Functions [1]: [partial_avg(netpaid#82)] @@ -440,10 +440,10 @@ Results [2]: [sum#85, count#86] Input [2]: [sum#85, count#86] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 4] +(79) CometNativeColumnarToRow Input [2]: [sum#85, count#86] -(80) HashAggregate [codegen id : 4] +(80) HashAggregate [codegen id : 3] Input [2]: [sum#85, count#86] Keys: [] Functions [1]: [avg(netpaid#82)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/extended.txt index 53a4c64a61..4c22965f0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/extended.txt @@ -1,19 +1,19 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Filter : +- Subquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometProject @@ -45,21 +45,21 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometProject @@ -91,7 +91,7 @@ CometColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/simplified.txt index b24e51723a..4758db93c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/simplified.txt @@ -1,64 +1,26 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] +CometNativeColumnarToRow + CometSort [c_last_name,c_first_name,s_store_name,paid] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (3) + Filter [paid] + Subquery #1 + WholeStageCodegen (3) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -68,32 +30,66 @@ WholeStageCodegen (5) CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + ReusedExchange [sr_item_sk,sr_ticket_number] #5 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + WholeStageCodegen (1) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + InputAdapter + CometNativeColumnarToRow + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometExchange [ss_ticket_number,ss_item_sk] #4 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometSort [sr_item_sk,sr_ticket_number] + CometExchange [sr_ticket_number,sr_item_sk] #5 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #9 + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/explain.txt index 54ebedf7d3..7d7e13c8d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/explain.txt @@ -45,7 +45,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -190,7 +190,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -208,24 +208,24 @@ Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isno Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#34, 2)) AS ca_state#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#35, 10)) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] (36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -236,17 +236,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] @@ -257,17 +257,17 @@ Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty# Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) @@ -279,7 +279,7 @@ Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(50) CometColumnarToRow [codegen id : 4] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== @@ -392,23 +392,23 @@ Arguments: [ss_customer_sk#52], [c_customer_sk#71], Inner, BuildRight Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] Arguments: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75], [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -(69) CometColumnarToRow [codegen id : 2] +(69) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] (70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] -(71) BroadcastHashJoin [codegen id : 2] +(71) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#72, c_birth_country#75, s_zip#61] Right keys [3]: [ca_address_sk#76, upper(ca_country#79), ca_zip#78] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(72) Project [codegen id : 1] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75, ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] -(73) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] @@ -419,17 +419,17 @@ Results [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_ Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] Arguments: hashpartitioning(c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometColumnarToRow [codegen id : 3] +(75) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] -(76) HashAggregate [codegen id : 3] +(76) HashAggregate [codegen id : 2] Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#82] -(77) HashAggregate [codegen id : 3] +(77) HashAggregate [codegen id : 2] Input [1]: [netpaid#82] Keys: [] Functions [1]: [partial_avg(netpaid#82)] @@ -440,10 +440,10 @@ Results [2]: [sum#85, count#86] Input [2]: [sum#85, count#86] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 4] +(79) CometColumnarToRow [codegen id : 3] Input [2]: [sum#85, count#86] -(80) HashAggregate [codegen id : 4] +(80) HashAggregate [codegen id : 3] Input [2]: [sum#85, count#86] Keys: [] Functions [1]: [avg(netpaid#82)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt index b9384ca04f..91b709aa47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt @@ -45,7 +45,7 @@ CometColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -91,7 +91,7 @@ CometColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/simplified.txt index 62f492f632..b657d3dae1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (5) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -47,13 +47,13 @@ WholeStageCodegen (5) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -91,9 +91,7 @@ WholeStageCodegen (5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt index 54ebedf7d3..7d7e13c8d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt @@ -45,7 +45,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -190,7 +190,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -208,24 +208,24 @@ Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isno Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#34, 2)) AS ca_state#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#35, 10)) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] (36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -236,17 +236,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] @@ -257,17 +257,17 @@ Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty# Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) @@ -279,7 +279,7 @@ Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(50) CometColumnarToRow [codegen id : 4] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== @@ -392,23 +392,23 @@ Arguments: [ss_customer_sk#52], [c_customer_sk#71], Inner, BuildRight Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] Arguments: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75], [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -(69) CometColumnarToRow [codegen id : 2] +(69) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] (70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] -(71) BroadcastHashJoin [codegen id : 2] +(71) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#72, c_birth_country#75, s_zip#61] Right keys [3]: [ca_address_sk#76, upper(ca_country#79), ca_zip#78] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(72) Project [codegen id : 1] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75, ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] -(73) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] @@ -419,17 +419,17 @@ Results [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_ Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] Arguments: hashpartitioning(c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometColumnarToRow [codegen id : 3] +(75) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] -(76) HashAggregate [codegen id : 3] +(76) HashAggregate [codegen id : 2] Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#82] -(77) HashAggregate [codegen id : 3] +(77) HashAggregate [codegen id : 2] Input [1]: [netpaid#82] Keys: [] Functions [1]: [partial_avg(netpaid#82)] @@ -440,10 +440,10 @@ Results [2]: [sum#85, count#86] Input [2]: [sum#85, count#86] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 4] +(79) CometColumnarToRow [codegen id : 3] Input [2]: [sum#85, count#86] -(80) HashAggregate [codegen id : 4] +(80) HashAggregate [codegen id : 3] Input [2]: [sum#85, count#86] Keys: [] Functions [1]: [avg(netpaid#82)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/extended.txt index b9384ca04f..91b709aa47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/extended.txt @@ -45,7 +45,7 @@ CometColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -91,7 +91,7 @@ CometColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt index 62f492f632..b657d3dae1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (5) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -47,13 +47,13 @@ WholeStageCodegen (5) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -91,9 +91,7 @@ WholeStageCodegen (5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/explain.txt index f025c0c7fd..44e499db34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (78) +- Union (77) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * Project (27) @@ -17,23 +17,23 @@ TakeOrderedAndProject (78) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : : +- ReusedExchange (11) : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) + : : +- CometNativeColumnarToRow (17) : : +- CometProject (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.store (14) : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) + : +- CometNativeColumnarToRow (24) : +- CometProject (23) : +- CometFilter (22) : +- CometNativeScan parquet spark_catalog.default.item (21) :- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) @@ -50,13 +50,13 @@ TakeOrderedAndProject (78) : : : : +- ReusedExchange (35) : : : +- ReusedExchange (38) : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) + : : +- CometNativeColumnarToRow (44) : : +- CometProject (43) : : +- CometFilter (42) : : +- CometNativeScan parquet spark_catalog.default.store (41) : +- ReusedExchange (48) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- CometColumnarExchange (74) +- * HashAggregate (73) +- * Project (72) @@ -74,7 +74,7 @@ TakeOrderedAndProject (78) : : +- ReusedExchange (61) : +- ReusedExchange (64) +- BroadcastExchange (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometFilter (68) +- CometNativeScan parquet spark_catalog.default.item (67) @@ -87,10 +87,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) @@ -109,33 +109,33 @@ Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) A Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -154,20 +154,20 @@ Condition : ((isnotnull(s_state#16) AND (static_invoke(CharVarcharCodegenUtils.r Input [2]: [s_store_sk#15, s_state#16] Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#16, 2)) AS s_state#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [s_store_sk#15, s_state#17] (18) BroadcastExchange Input [2]: [s_store_sk#15, s_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] @@ -186,24 +186,24 @@ Condition : isnotnull(i_item_sk#18) Input [2]: [i_item_sk#18, i_item_id#19] Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#19, 16)) AS i_item_id#20] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [2]: [i_item_sk#18, i_item_id#20] (25) BroadcastExchange Input [2]: [i_item_sk#18, i_item_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] Keys [2]: [i_item_id#21, s_state#22] Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] @@ -214,10 +214,10 @@ Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] Keys [2]: [i_item_id#21, s_state#22] Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] @@ -232,36 +232,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 11] +(33) ColumnarToRow [codegen id : 3] Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -(34) Filter [codegen id : 11] +(34) Filter [codegen id : 3] Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) (35) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#60] -(36) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_cdemo_sk#53] Right keys [1]: [cd_demo_sk#60] Join type: Inner Join condition: None -(37) Project [codegen id : 11] +(37) Project [codegen id : 3] Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#60] (38) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#61] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#59] Right keys [1]: [d_date_sk#61] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 3] Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#61] @@ -280,37 +280,37 @@ Condition : ((isnotnull(s_state#63) AND (static_invoke(CharVarcharCodegenUtils.r Input [2]: [s_store_sk#62, s_state#63] Arguments: [s_store_sk#62], [s_store_sk#62] -(44) CometColumnarToRow [codegen id : 9] +(44) CometNativeColumnarToRow Input [1]: [s_store_sk#62] (45) BroadcastExchange Input [1]: [s_store_sk#62] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(46) BroadcastHashJoin [codegen id : 11] +(46) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#54] Right keys [1]: [s_store_sk#62] Join type: Inner Join condition: None -(47) Project [codegen id : 11] +(47) Project [codegen id : 3] Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#62] (48) ReusedExchange [Reuses operator id: 25] Output [2]: [i_item_sk#64, i_item_id#65] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#52] Right keys [1]: [i_item_sk#64] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 3] Output [5]: [i_item_id#65, ss_quantity#55 AS agg1#66, ss_list_price#56 AS agg2#67, ss_coupon_amt#58 AS agg3#68, ss_sales_price#57 AS agg4#69] Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#64, i_item_id#65] -(51) HashAggregate [codegen id : 11] +(51) HashAggregate [codegen id : 3] Input [5]: [i_item_id#65, agg1#66, agg2#67, agg3#68, agg4#69] Keys [1]: [i_item_id#65] Functions [4]: [partial_avg(agg1#66), partial_avg(UnscaledValue(agg2#67)), partial_avg(UnscaledValue(agg3#68)), partial_avg(UnscaledValue(agg4#69))] @@ -321,10 +321,10 @@ Results [9]: [i_item_id#65, sum#78, count#79, sum#80, count#81, sum#82, count#83 Input [9]: [i_item_id#65, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] Arguments: hashpartitioning(i_item_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(53) CometColumnarToRow [codegen id : 12] +(53) CometNativeColumnarToRow Input [9]: [i_item_id#65, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] -(54) HashAggregate [codegen id : 12] +(54) HashAggregate [codegen id : 4] Input [9]: [i_item_id#65, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] Keys [1]: [i_item_id#65] Functions [4]: [avg(agg1#66), avg(UnscaledValue(agg2#67)), avg(UnscaledValue(agg3#68)), avg(UnscaledValue(agg4#69))] @@ -339,49 +339,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_s PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 17] +(56) ColumnarToRow [codegen id : 5] Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -(57) Filter [codegen id : 17] +(57) Filter [codegen id : 5] Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#104] -(59) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#97] Right keys [1]: [cd_demo_sk#104] Join type: Inner Join condition: None -(60) Project [codegen id : 17] +(60) Project [codegen id : 5] Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#105] -(62) BroadcastHashJoin [codegen id : 17] +(62) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#103] Right keys [1]: [d_date_sk#105] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(63) Project [codegen id : 5] Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] (64) ReusedExchange [Reuses operator id: 45] Output [1]: [s_store_sk#106] -(65) BroadcastHashJoin [codegen id : 17] +(65) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#98] Right keys [1]: [s_store_sk#106] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(66) Project [codegen id : 5] Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] @@ -396,24 +396,24 @@ ReadSchema: struct Input [1]: [i_item_sk#107] Condition : isnotnull(i_item_sk#107) -(69) CometColumnarToRow [codegen id : 16] +(69) CometNativeColumnarToRow Input [1]: [i_item_sk#107] (70) BroadcastExchange Input [1]: [i_item_sk#107] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(71) BroadcastHashJoin [codegen id : 17] +(71) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#96] Right keys [1]: [i_item_sk#107] Join type: Inner Join condition: None -(72) Project [codegen id : 17] +(72) Project [codegen id : 5] Output [4]: [ss_quantity#99 AS agg1#108, ss_list_price#100 AS agg2#109, ss_coupon_amt#102 AS agg3#110, ss_sales_price#101 AS agg4#111] Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] -(73) HashAggregate [codegen id : 17] +(73) HashAggregate [codegen id : 5] Input [4]: [agg1#108, agg2#109, agg3#110, agg4#111] Keys: [] Functions [4]: [partial_avg(agg1#108), partial_avg(UnscaledValue(agg2#109)), partial_avg(UnscaledValue(agg3#110)), partial_avg(UnscaledValue(agg4#111))] @@ -424,10 +424,10 @@ Results [8]: [sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#12 Input [8]: [sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(75) CometColumnarToRow [codegen id : 18] +(75) CometNativeColumnarToRow Input [8]: [sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] -(76) HashAggregate [codegen id : 18] +(76) HashAggregate [codegen id : 6] Input [8]: [sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] Keys: [] Functions [4]: [avg(agg1#108), avg(UnscaledValue(agg2#109)), avg(UnscaledValue(agg3#110)), avg(UnscaledValue(agg4#111))] @@ -444,7 +444,7 @@ Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometNativeScan parquet spark_catalog.default.date_dim (79) @@ -465,7 +465,7 @@ Condition : ((isnotnull(d_year#139) AND (d_year#139 = 1998)) AND isnotnull(d_dat Input [2]: [d_date_sk#14, d_year#139] Arguments: [d_date_sk#14], [d_date_sk#14] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt index 9926d14ae2..4eec8b0953 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -17,32 +17,32 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,27 +58,27 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -94,22 +94,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/simplified.txt index c7093301af..2dad05ae3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,s_state] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -21,44 +21,36 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (11) + WholeStageCodegen (3) HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -79,20 +71,18 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) + WholeStageCodegen (6) HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #8 - WholeStageCodegen (17) + WholeStageCodegen (5) HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -115,8 +105,6 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ReusedExchange [s_store_sk] #7 InputAdapter BroadcastExchange #9 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/explain.txt index bcba275572..d096bdf047 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/explain.txt @@ -402,7 +402,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -423,7 +423,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt index affbc9ef37..a3e6b17491 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/simplified.txt index 0d18ca5626..d24cb217a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt index bcba275572..d096bdf047 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt @@ -402,7 +402,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -423,7 +423,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/extended.txt index affbc9ef37..a3e6b17491 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0d18ca5626..d24cb217a9 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/explain.txt index 243b950bc1..fb3e4db962 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometSort (34) +- CometColumnarExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) :- * Filter (25) : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -20,17 +20,17 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.customer (26) @@ -44,23 +44,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] @@ -79,20 +79,20 @@ Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND is Input [2]: [s_store_sk#8, s_county#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (11) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] @@ -111,24 +111,24 @@ Condition : ((((isnotnull(hd_vehicle_count#13) AND ((static_invoke(CharVarcharCo Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#10] (18) BroadcastExchange Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] @@ -139,17 +139,17 @@ Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) @@ -168,20 +168,20 @@ Condition : isnotnull(c_customer_sk#18) Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] (30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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] @@ -193,14 +193,14 @@ Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC 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: [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 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] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (40) -+- * CometColumnarToRow (39) ++- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometNativeScan parquet spark_catalog.default.date_dim (36) @@ -221,7 +221,7 @@ Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(39) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (40) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt index 9e4596c12c..40230582bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- BroadcastHashJoin :- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,27 +19,27 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/simplified.txt index 63c6ac3645..bf781c83d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/simplified.txt @@ -1,59 +1,49 @@ -WholeStageCodegen (7) - 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 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] +CometNativeColumnarToRow + 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 + WholeStageCodegen (2) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/explain.txt index 94be39468a..39ca1dd35f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/simplified.txt index d57afd4fcf..ee10f76856 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 94be39468a..39ca1dd35f 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 @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d57afd4fcf..ee10f76856 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/explain.txt index 7e28922e9f..f90f03a46f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -13,7 +13,7 @@ TakeOrderedAndProject (45) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -35,12 +35,12 @@ TakeOrderedAndProject (45) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_address (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) @@ -57,7 +57,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] (6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] @@ -87,7 +87,7 @@ Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi @@ -100,19 +100,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] (13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#12] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] @@ -120,7 +120,7 @@ Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) @@ -133,19 +133,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#14] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#13] Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] @@ -153,17 +153,17 @@ Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] Input [1]: [cs_ship_customer_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#13] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(25) Filter [codegen id : 4] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -182,20 +182,20 @@ Condition : isnotnull(ca_address_sk#16) Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) AS ca_state#18] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [2]: [ca_address_sk#16, ca_state#18] (31) BroadcastExchange Input [2]: [ca_address_sk#16, ca_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, ca_state#18] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#18] @@ -214,24 +214,24 @@ Condition : isnotnull(cd_demo_sk#19) Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#20, 1)) AS cd_gender#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] (38) BroadcastExchange Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 4] Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Input [8]: [c_current_cdemo_sk#4, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 4] Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#22), partial_max(cd_dep_count#22), partial_sum(cd_dep_count#22), partial_avg(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_sum(cd_dep_employed_count#23), partial_avg(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_sum(cd_dep_college_count#24)] @@ -242,10 +242,10 @@ Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 5] Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [count(1), avg(cd_dep_count#22), max(cd_dep_count#22), sum(cd_dep_count#22), avg(cd_dep_employed_count#23), max(cd_dep_employed_count#23), sum(cd_dep_employed_count#23), avg(cd_dep_college_count#24), max(cd_dep_college_count#24), sum(cd_dep_college_count#24)] @@ -260,7 +260,7 @@ Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * CometColumnarToRow (49) ++- CometNativeColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometNativeScan parquet spark_catalog.default.date_dim (46) @@ -281,7 +281,7 @@ Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 2 Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt index 8935b6cc13..40a8ef4a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -22,12 +22,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,17 +49,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/simplified.txt index 0fb95102c3..e7163949cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -29,17 +29,15 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,7 +48,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -61,17 +59,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/explain.txt index c8f5d42144..070ed8f89a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#20) Input [2]: [ca_address_sk#20, ca_state#21] Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#20, ca_state#22] (33) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#22] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#23) Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/simplified.txt index fea6a32741..af19f84961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt index c8f5d42144..070ed8f89a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#16] Input [1]: [cs_ship_customer_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#16] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#20) Input [2]: [ca_address_sk#20, ca_state#21] Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#20, ca_state#22] (33) BroadcastExchange Input [2]: [ca_address_sk#20, ca_state#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#22] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#23) Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (40) BroadcastExchange Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#23] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt index fea6a32741..af19f84961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/explain.txt index bfbcea798a..058ceb3bcc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -11,7 +11,7 @@ TakeOrderedAndProject (43) : :- * Project (24) : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) + : : : :- CometNativeColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : +- BroadcastExchange (9) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- Scan parquet spark_catalog.default.catalog_sales (16) : : +- ReusedExchange (18) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer_address (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) @@ -55,7 +55,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] (6) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] @@ -85,7 +85,7 @@ Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi @@ -98,19 +98,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] (13) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#10] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#8 AS customsk#11] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] @@ -121,19 +121,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] +(17) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] (18) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#14] -(19) BroadcastHashJoin [codegen id : 6] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(20) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#12 AS customsk#15] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] @@ -143,13 +143,13 @@ Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] Input [1]: [customsk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#11] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 9] +(24) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -168,20 +168,20 @@ Condition : isnotnull(ca_address_sk#16) Input [2]: [ca_address_sk#16, ca_state#17] Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) AS ca_state#18] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [2]: [ca_address_sk#16, ca_state#18] (29) BroadcastExchange Input [2]: [ca_address_sk#16, ca_state#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#16] Join type: Inner Join condition: None -(31) Project [codegen id : 9] +(31) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, ca_state#18] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16, ca_state#18] @@ -200,24 +200,24 @@ Condition : isnotnull(cd_demo_sk#19) Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#20, 1)) AS cd_gender#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(35) CometColumnarToRow [codegen id : 8] +(35) CometNativeColumnarToRow Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] (36) BroadcastExchange Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#19] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 4] Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Input [8]: [c_current_cdemo_sk#2, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -(39) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 4] Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#22), partial_max(cd_dep_count#22), partial_sum(cd_dep_count#22), partial_avg(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_sum(cd_dep_employed_count#23), partial_avg(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_sum(cd_dep_college_count#24)] @@ -228,10 +228,10 @@ Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 5] Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] Functions [10]: [count(1), avg(cd_dep_count#22), max(cd_dep_count#22), sum(cd_dep_count#22), avg(cd_dep_employed_count#23), max(cd_dep_employed_count#23), sum(cd_dep_employed_count#23), avg(cd_dep_college_count#24), max(cd_dep_college_count#24), sum(cd_dep_college_count#24)] @@ -246,7 +246,7 @@ Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -267,7 +267,7 @@ Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 1 Input [3]: [d_date_sk#7, d_year#75, d_qoy#76] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt index 8aebe6be9a..9091536e96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : :- Project : : +- BroadcastHashJoin : : :- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange @@ -20,12 +20,12 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,7 +37,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -47,17 +47,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/simplified.txt index bed891a0a3..4fe5cb9c56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,13 +13,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -27,18 +27,16 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 Union - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -47,7 +45,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -58,17 +56,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/explain.txt index e4756635dc..3d57058f0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/explain.txt @@ -232,7 +232,7 @@ Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, c Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999 Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/simplified.txt index 69e8d4868c..15668eeef0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt index e4756635dc..3d57058f0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt @@ -232,7 +232,7 @@ Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, c Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999 Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 69e8d4868c..15668eeef0 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/explain.txt index 19961d7852..327c271a5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- CometExchange (43) +- CometHashAggregate (42) @@ -10,7 +10,7 @@ TakeOrderedAndProject (48) +- * HashAggregate (40) +- Union (39) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -24,28 +24,28 @@ TakeOrderedAndProject (48) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.item (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.store (14) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) + : +- CometNativeColumnarToRow (26) : +- ReusedExchange (25) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * HashAggregate (34) - +- * CometColumnarToRow (33) + +- CometNativeColumnarToRow (33) +- ReusedExchange (32) @@ -57,23 +57,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (4) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] @@ -92,20 +92,20 @@ Condition : isnotnull(i_item_sk#8) Input [3]: [i_item_sk#8, i_class#9, i_category#10] Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#12] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#8, i_class#11, i_category#12] (11) BroadcastExchange Input [3]: [i_item_sk#8, i_class#11, i_category#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] @@ -124,24 +124,24 @@ Condition : ((isnotnull(s_state#14) AND (static_invoke(CharVarcharCodegenUtils.r Input [2]: [s_store_sk#13, s_state#14] Arguments: [s_store_sk#13], [s_store_sk#13] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [s_store_sk#13] (18) BroadcastExchange Input [1]: [s_store_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Keys [2]: [i_category#12, i_class#11] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -152,10 +152,10 @@ Results [4]: [i_category#12, i_class#11, sum#17, sum#18] Input [4]: [i_category#12, i_class#11, sum#17, sum#18] Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 5] +(23) CometNativeColumnarToRow Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [4]: [i_category#12, i_class#11, sum#17, sum#18] Keys [2]: [i_category#12, i_class#11] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] @@ -165,17 +165,17 @@ Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / M (25) ReusedExchange [Reuses operator id: 22] Output [4]: [i_category#27, i_class#28, sum#29, sum#30] -(26) CometColumnarToRow [codegen id : 10] +(26) CometNativeColumnarToRow Input [4]: [i_category#27, i_class#28, sum#29, sum#30] -(27) HashAggregate [codegen id : 10] +(27) HashAggregate [codegen id : 4] Input [4]: [i_category#27, i_class#28, sum#29, sum#30] Keys [2]: [i_category#27, i_class#28] Functions [2]: [sum(UnscaledValue(ss_net_profit#31)), sum(UnscaledValue(ss_ext_sales_price#32))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#31))#33, sum(UnscaledValue(ss_ext_sales_price#32))#34] Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#31))#33,17,2) AS ss_net_profit#35, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#32))#34,17,2) AS ss_ext_sales_price#36, i_category#27] -(28) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 4] Input [3]: [ss_net_profit#35, ss_ext_sales_price#36, i_category#27] Keys [1]: [i_category#27] Functions [2]: [partial_sum(ss_net_profit#35), partial_sum(ss_ext_sales_price#36)] @@ -186,10 +186,10 @@ Results [5]: [i_category#27, sum#41, isEmpty#42, sum#43, isEmpty#44] Input [5]: [i_category#27, sum#41, isEmpty#42, sum#43, isEmpty#44] Arguments: hashpartitioning(i_category#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 11] +(30) CometNativeColumnarToRow Input [5]: [i_category#27, sum#41, isEmpty#42, sum#43, isEmpty#44] -(31) HashAggregate [codegen id : 11] +(31) HashAggregate [codegen id : 5] Input [5]: [i_category#27, sum#41, isEmpty#42, sum#43, isEmpty#44] Keys [1]: [i_category#27] Functions [2]: [sum(ss_net_profit#35), sum(ss_ext_sales_price#36)] @@ -199,17 +199,17 @@ Results [6]: [(sum(ss_net_profit#35)#45 / sum(ss_ext_sales_price#36)#46) AS gros (32) ReusedExchange [Reuses operator id: 22] Output [4]: [i_category#52, i_class#53, sum#54, sum#55] -(33) CometColumnarToRow [codegen id : 16] +(33) CometNativeColumnarToRow Input [4]: [i_category#52, i_class#53, sum#54, sum#55] -(34) HashAggregate [codegen id : 16] +(34) HashAggregate [codegen id : 7] Input [4]: [i_category#52, i_class#53, sum#54, sum#55] Keys [2]: [i_category#52, i_class#53] Functions [2]: [sum(UnscaledValue(ss_net_profit#56)), sum(UnscaledValue(ss_ext_sales_price#57))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#56))#33, sum(UnscaledValue(ss_ext_sales_price#57))#34] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#33,17,2) AS ss_net_profit#58, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#57))#34,17,2) AS ss_ext_sales_price#59] -(35) HashAggregate [codegen id : 16] +(35) HashAggregate [codegen id : 7] Input [2]: [ss_net_profit#58, ss_ext_sales_price#59] Keys: [] Functions [2]: [partial_sum(ss_net_profit#58), partial_sum(ss_ext_sales_price#59)] @@ -220,10 +220,10 @@ Results [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 17] +(37) CometNativeColumnarToRow Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] -(38) HashAggregate [codegen id : 17] +(38) HashAggregate [codegen id : 8] Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] Keys: [] Functions [2]: [sum(ss_net_profit#58), sum(ss_ext_sales_price#59)] @@ -232,7 +232,7 @@ Results [6]: [(sum(ss_net_profit#58)#68 / sum(ss_ext_sales_price#59)#69) AS gros (39) Union -(40) HashAggregate [codegen id : 18] +(40) HashAggregate [codegen id : 9] Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] Functions: [] @@ -256,14 +256,14 @@ Arguments: hashpartitioning(lochierarchy#26, _w0#76, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76] Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76], [lochierarchy#26 ASC NULLS FIRST, _w0#76 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 19] +(45) CometNativeColumnarToRow Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76] (46) Window Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76] Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#76, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#77], [lochierarchy#26, _w0#76], [gross_margin#21 ASC NULLS FIRST] -(47) Project [codegen id : 20] +(47) Project [codegen id : 10] Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#77] Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76, rank_within_parent#77] @@ -275,7 +275,7 @@ Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -296,7 +296,7 @@ Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#7, d_year#78] Arguments: [d_date_sk#7], [d_date_sk#7] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt index 7d9e9883d2..1fc86dc5a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -9,7 +9,7 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -23,31 +23,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -61,31 +61,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -99,22 +99,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/simplified.txt index 65728fc6f2..a101d9fb6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/simplified.txt @@ -1,81 +1,73 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (20) + WholeStageCodegen (10) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - 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 (18) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + 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 (9) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (5) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category] #7 + WholeStageCodegen (4) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (8) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #8 + WholeStageCodegen (7) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt index 7233391f5e..69991e6176 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt @@ -244,7 +244,7 @@ Arguments: 100, [lochierarchy#47 DESC NULLS LAST, CASE WHEN (lochierarchy#47 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -265,7 +265,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt index 1f6c984b22..968ea0611f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,7 +53,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -85,7 +85,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt index e91b278c14..9fddc4902a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt @@ -25,12 +25,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt index 7233391f5e..69991e6176 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt @@ -244,7 +244,7 @@ Arguments: 100, [lochierarchy#47 DESC NULLS LAST, CASE WHEN (lochierarchy#47 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -265,7 +265,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt index 1f6c984b22..968ea0611f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,7 +53,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -85,7 +85,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e91b278c14..9fddc4902a 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 @@ -25,12 +25,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/explain.txt index 490593f3bf..3f4a4ce43b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.store_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.store (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCode Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.store_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#6] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] (17) BroadcastExchange Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#7] Right keys [1]: [s_store_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] @@ -151,10 +151,10 @@ Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_yea Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] (27) Window Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] (32) ReusedExchange [Reuses operator id: 21] Output [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] Keys [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29] Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#24, i_brand#25, s_store_name#26, s_compan Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] Arguments: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32], [i_category#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, s_store_name#26 ASC NULLS FIRST, s_company_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] (38) Window Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#24, i_brand#25, s_store_name#26, s_company_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] Input [8]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32, rn#33] @@ -228,13 +228,13 @@ Input [8]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_yea Input [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, (rn#33 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#32] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] @@ -245,14 +245,14 @@ Output [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_ye Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] Arguments: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40], [i_category#34 ASC NULLS FIRST, i_brand#35 ASC NULLS FIRST, s_store_name#36 ASC NULLS FIRST, s_company_name#37 ASC NULLS FIRST, d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] (46) Window Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] Arguments: [rank(d_year#38, d_moy#39) windowspecdefinition(i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#34, i_brand#35, s_store_name#36, s_company_name#37], [d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [6]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] Input [8]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40, rn#41] @@ -260,13 +260,13 @@ Input [8]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_yea Input [6]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, (rn#41 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#32 AS psum#42, sum_sales#40 AS nsum#43] Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#32, i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt index 092083e465..ab44bbef5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/simplified.txt index ee8e4dd40c..404ef436b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk,s_store_name,s_company_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt index bf8cb9a216..953c0a2506 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt index a62c33ecc1..8a10571eff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt index bf8cb9a216..953c0a2506 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 a62c33ecc1..8a10571eff 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/explain.txt index 070113d8ea..9f67fa1915 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (84) +CometNativeColumnarToRow (84) +- CometTakeOrderedAndProject (83) +- CometHashAggregate (82) +- CometColumnarExchange (81) @@ -10,11 +10,11 @@ : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometSort (20) : +- CometColumnarExchange (19) : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometColumnarExchange (16) : +- * HashAggregate (15) : +- * Project (14) @@ -26,7 +26,7 @@ : : : +- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) + : : +- CometNativeColumnarToRow (9) : : +- CometProject (8) : : +- CometFilter (7) : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) @@ -36,11 +36,11 @@ : +- Window (50) : +- * Sort (49) : +- Window (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometSort (46) : +- CometColumnarExchange (45) : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -52,7 +52,7 @@ : : : +- * Filter (29) : : : +- * ColumnarToRow (28) : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) + : : +- CometNativeColumnarToRow (35) : : +- CometProject (34) : : +- CometFilter (33) : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) @@ -62,11 +62,11 @@ +- Window (76) +- * Sort (75) +- Window (74) - +- * CometColumnarToRow (73) + +- CometNativeColumnarToRow (73) +- CometSort (72) +- CometColumnarExchange (71) +- * HashAggregate (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometColumnarExchange (68) +- * HashAggregate (67) +- * Project (66) @@ -78,7 +78,7 @@ : : +- * Filter (55) : : +- * ColumnarToRow (54) : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) + : +- CometNativeColumnarToRow (61) : +- CometProject (60) : +- CometFilter (59) : +- CometNativeScan parquet spark_catalog.default.store_returns (58) @@ -123,33 +123,33 @@ Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AN Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometColumnarToRow +(9) CometNativeColumnarToRow Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [2]: [ws_order_number#2, ws_item_sk#1] Right keys [2]: [wr_order_number#9, wr_item_sk#8] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] (12) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#13] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#6] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 2] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -160,10 +160,10 @@ Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty# Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -178,14 +178,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 5] +(21) CometNativeColumnarToRow Input [3]: [item#30, return_ratio#31, currency_ratio#32] (22) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 4] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 @@ -193,11 +193,11 @@ Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 5] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(26) Project [codegen id : 7] +(26) Project [codegen id : 5] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -209,14 +209,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -(29) Filter [codegen id : 8] +(29) Filter [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(30) Project [codegen id : 8] +(30) Project [codegen id : 6] Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] @@ -239,33 +239,33 @@ Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.0 Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(35) CometColumnarToRow +(35) CometNativeColumnarToRow Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 7] Left keys [2]: [cs_order_number#37, cs_item_sk#36] Right keys [2]: [cr_order_number#43, cr_item_sk#42] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 7] Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] (38) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#47] -(39) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#47] Join type: Inner Join condition: None -(40) Project [codegen id : 10] +(40) Project [codegen id : 7] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] -(41) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -276,10 +276,10 @@ Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometColumnarToRow [codegen id : 11] +(43) CometNativeColumnarToRow Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 8] Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -294,14 +294,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 12] +(47) CometNativeColumnarToRow Input [3]: [item#64, return_ratio#65, currency_ratio#66] (48) Window Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] -(49) Sort [codegen id : 13] +(49) Sort [codegen id : 9] Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 @@ -309,11 +309,11 @@ Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] -(51) Filter [codegen id : 14] +(51) Filter [codegen id : 10] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) -(52) Project [codegen id : 14] +(52) Project [codegen id : 10] Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] @@ -325,14 +325,14 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 15] +(54) ColumnarToRow [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -(55) Filter [codegen id : 15] +(55) Filter [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) -(56) Project [codegen id : 15] +(56) Project [codegen id : 11] Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] @@ -355,33 +355,33 @@ Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AN Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(61) CometColumnarToRow +(61) CometNativeColumnarToRow Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(62) BroadcastHashJoin [codegen id : 17] +(62) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(63) Project [codegen id : 12] Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] (64) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#81] -(65) BroadcastHashJoin [codegen id : 17] +(65) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ss_sold_date_sk#75] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(66) Project [codegen id : 12] Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] -(67) HashAggregate [codegen id : 17] +(67) HashAggregate [codegen id : 12] Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Keys [1]: [ss_item_sk#70] Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -392,10 +392,10 @@ Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(69) CometColumnarToRow [codegen id : 18] +(69) CometNativeColumnarToRow Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -(70) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 13] Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Keys [1]: [ss_item_sk#70] Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -410,14 +410,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [3]: [item#98, return_ratio#99, currency_ratio#100] (74) Window Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] -(75) Sort [codegen id : 20] +(75) Sort [codegen id : 14] Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 @@ -425,17 +425,17 @@ Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] -(77) Filter [codegen id : 21] +(77) Filter [codegen id : 15] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) -(78) Project [codegen id : 21] +(78) Project [codegen id : 15] Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] (79) Union -(80) HashAggregate [codegen id : 22] +(80) HashAggregate [codegen id : 16] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] @@ -455,14 +455,14 @@ Functions: [] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) CometColumnarToRow [codegen id : 23] +(84) CometNativeColumnarToRow Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometNativeScan parquet spark_catalog.default.date_dim (85) @@ -483,7 +483,7 @@ Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 Input [3]: [d_date_sk#13, d_year#104, d_moy#105] Arguments: [d_date_sk#13], [d_date_sk#13] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt index 6c4ded1515..0090739d55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -9,11 +9,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -27,16 +27,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -45,11 +45,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -62,12 +62,12 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -76,11 +76,11 @@ CometColumnarToRow +- Window +- Sort +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -93,12 +93,12 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/simplified.txt index 047269434d..018aec1d47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/simplified.txt @@ -1,140 +1,130 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + CometNativeColumnarToRow + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #6 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [cs_item_sk] #7 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (15) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #9 + WholeStageCodegen (13) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (12) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (11) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt index 74702d596d..b027d714df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt index ca80833ee5..3353e1c331 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt index 74702d596d..b027d714df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ca80833ee5..3353e1c331 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 @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/explain.txt index 35eddeb2fd..ca2e451d67 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (77) +- * Project (73) +- * BroadcastHashJoin Inner BuildRight (72) :- Window (65) - : +- * CometColumnarToRow (64) + : +- CometNativeColumnarToRow (64) : +- CometSort (63) : +- CometExchange (62) : +- CometProject (61) @@ -15,18 +15,18 @@ TakeOrderedAndProject (77) : :- CometSort (29) : : +- CometColumnarExchange (28) : : +- * HashAggregate (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometColumnarExchange (25) : : +- * HashAggregate (24) : : +- * Project (23) : : +- * BroadcastHashJoin Inner BuildRight (22) : : :- * Project (15) : : : +- Window (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometSort (12) : : : +- CometColumnarExchange (11) : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometColumnarExchange (8) : : : +- * HashAggregate (7) : : : +- * Project (6) @@ -38,24 +38,24 @@ TakeOrderedAndProject (77) : : +- BroadcastExchange (21) : : +- * Project (20) : : +- Window (19) - : : +- * CometColumnarToRow (18) + : : +- CometNativeColumnarToRow (18) : : +- CometSort (17) : : +- ReusedExchange (16) : +- CometSort (58) : +- CometColumnarExchange (57) : +- * HashAggregate (56) - : +- * CometColumnarToRow (55) + : +- CometNativeColumnarToRow (55) : +- CometColumnarExchange (54) : +- * HashAggregate (53) : +- * Project (52) : +- * BroadcastHashJoin Inner BuildRight (51) : :- * Project (44) : : +- Window (43) - : : +- * CometColumnarToRow (42) + : : +- CometNativeColumnarToRow (42) : : +- CometSort (41) : : +- CometColumnarExchange (40) : : +- * HashAggregate (39) - : : +- * CometColumnarToRow (38) + : : +- CometNativeColumnarToRow (38) : : +- CometColumnarExchange (37) : : +- * HashAggregate (36) : : +- * Project (35) @@ -67,13 +67,13 @@ TakeOrderedAndProject (77) : +- BroadcastExchange (50) : +- * Project (49) : +- Window (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometSort (46) : +- ReusedExchange (45) +- BroadcastExchange (71) +- * Project (70) +- Window (69) - +- * CometColumnarToRow (68) + +- CometNativeColumnarToRow (68) +- CometSort (67) +- ReusedExchange (66) @@ -86,27 +86,27 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 82] Output [2]: [d_date_sk#5, d_date#6] -(5) BroadcastHashJoin [codegen id : 2] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 2] +(6) Project [codegen id : 1] Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -(7) HashAggregate [codegen id : 2] +(7) HashAggregate [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] @@ -117,10 +117,10 @@ Results [3]: [ws_item_sk#1, d_date#6, sum#8] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 3] +(9) CometNativeColumnarToRow Input [3]: [ws_item_sk#1, d_date#6, sum#8] -(10) HashAggregate [codegen id : 3] +(10) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] @@ -135,14 +135,14 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 4] +(13) CometNativeColumnarToRow Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] (14) Window Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 10] +(15) Project [codegen id : 6] Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] @@ -153,14 +153,14 @@ Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16], [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST] -(18) CometColumnarToRow [codegen id : 8] +(18) CometNativeColumnarToRow Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] (19) Window Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] -(20) Project [codegen id : 9] +(20) Project [codegen id : 5] Output [3]: [item_sk#13, sumws#15, rk#17] Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] @@ -168,17 +168,17 @@ Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] Input [3]: [item_sk#13, sumws#15, rk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 10] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_sk#10] Right keys [1]: [item_sk#13] Join type: Inner Join condition: (rk#12 >= rk#17) -(23) Project [codegen id : 10] +(23) Project [codegen id : 6] Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] -(24) HashAggregate [codegen id : 10] +(24) HashAggregate [codegen id : 6] Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [partial_sum(sumws#15)] @@ -189,10 +189,10 @@ Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometColumnarToRow [codegen id : 11] +(26) CometNativeColumnarToRow Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(27) HashAggregate [codegen id : 11] +(27) HashAggregate [codegen id : 7] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [sum(sumws#15)] @@ -215,27 +215,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 13] +(31) ColumnarToRow [codegen id : 8] Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -(32) Filter [codegen id : 13] +(32) Filter [codegen id : 8] Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#24) (33) ReusedExchange [Reuses operator id: 82] Output [2]: [d_date_sk#27, d_date#28] -(34) BroadcastHashJoin [codegen id : 13] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(35) Project [codegen id : 13] +(35) Project [codegen id : 8] Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27, d_date#28] -(36) HashAggregate [codegen id : 13] +(36) HashAggregate [codegen id : 8] Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] Keys [2]: [ss_item_sk#24, d_date#28] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] @@ -246,10 +246,10 @@ Results [3]: [ss_item_sk#24, d_date#28, sum#30] Input [3]: [ss_item_sk#24, d_date#28, sum#30] Arguments: hashpartitioning(ss_item_sk#24, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(38) CometColumnarToRow [codegen id : 14] +(38) CometNativeColumnarToRow Input [3]: [ss_item_sk#24, d_date#28, sum#30] -(39) HashAggregate [codegen id : 14] +(39) HashAggregate [codegen id : 9] Input [3]: [ss_item_sk#24, d_date#28, sum#30] Keys [2]: [ss_item_sk#24, d_date#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] @@ -264,14 +264,14 @@ Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumna Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] Arguments: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(42) CometColumnarToRow [codegen id : 15] +(42) CometNativeColumnarToRow Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] (43) Window Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#28 ASC NULLS FIRST] -(44) Project [codegen id : 21] +(44) Project [codegen id : 13] Output [4]: [item_sk#32, d_date#28, sumss#33, rk#34] Input [5]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24, rk#34] @@ -282,14 +282,14 @@ Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] Arguments: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38], [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 19] +(47) CometNativeColumnarToRow Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] (48) Window Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] -(49) Project [codegen id : 20] +(49) Project [codegen id : 12] Output [3]: [item_sk#35, sumss#37, rk#39] Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] @@ -297,17 +297,17 @@ Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] Input [3]: [item_sk#35, sumss#37, rk#39] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 21] +(51) BroadcastHashJoin [codegen id : 13] Left keys [1]: [item_sk#32] Right keys [1]: [item_sk#35] Join type: Inner Join condition: (rk#34 >= rk#39) -(52) Project [codegen id : 21] +(52) Project [codegen id : 13] Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#37] Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] -(53) HashAggregate [codegen id : 21] +(53) HashAggregate [codegen id : 13] Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#37] Keys [3]: [item_sk#32, d_date#28, sumss#33] Functions [1]: [partial_sum(sumss#37)] @@ -318,10 +318,10 @@ Results [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Arguments: hashpartitioning(item_sk#32, d_date#28, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(55) CometColumnarToRow [codegen id : 22] +(55) CometNativeColumnarToRow Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] -(56) HashAggregate [codegen id : 22] +(56) HashAggregate [codegen id : 14] Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Keys [3]: [item_sk#32, d_date#28, sumss#33] Functions [1]: [sum(sumss#37)] @@ -357,7 +357,7 @@ Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] -(64) CometColumnarToRow [codegen id : 23] +(64) CometNativeColumnarToRow Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] (65) Window @@ -371,14 +371,14 @@ Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] Arguments: [item_sk#51, d_date#52, web_sales#53, store_sales#54], [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST] -(68) CometColumnarToRow [codegen id : 46] +(68) CometNativeColumnarToRow Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] (69) Window Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] -(70) Project [codegen id : 47] +(70) Project [codegen id : 29] Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] @@ -386,31 +386,31 @@ Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] Input [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(72) BroadcastHashJoin [codegen id : 48] +(72) BroadcastHashJoin [codegen id : 30] Left keys [1]: [item_sk#46] Right keys [1]: [item_sk#51] Join type: Inner Join condition: (rk#50 >= rk#55) -(73) Project [codegen id : 48] +(73) Project [codegen id : 30] Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] -(74) HashAggregate [codegen id : 48] +(74) HashAggregate [codegen id : 30] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] Aggregate Attributes [2]: [max#56, max#57] Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] -(75) HashAggregate [codegen id : 48] +(75) HashAggregate [codegen id : 30] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [max(web_sales#53), max(store_sales#54)] Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] -(76) Filter [codegen id : 48] +(76) Filter [codegen id : 30] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) @@ -422,7 +422,7 @@ Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (82) -+- * CometColumnarToRow (81) ++- CometNativeColumnarToRow (81) +- CometProject (80) +- CometFilter (79) +- CometNativeScan parquet spark_catalog.default.date_dim (78) @@ -443,7 +443,7 @@ Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_mo Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(81) CometColumnarToRow [codegen id : 1] +(81) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (82) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt index 8eb3f067e2..92ee33eda6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject +- Project +- BroadcastHashJoin :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometExchange : +- CometProject @@ -14,18 +14,18 @@ TakeOrderedAndProject : :- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometSort : : : +- CometColumnarExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -35,23 +35,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -61,30 +61,30 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -94,18 +94,18 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -115,14 +115,14 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -131,18 +131,18 @@ TakeOrderedAndProject :- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -152,23 +152,23 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -178,30 +178,30 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -211,18 +211,18 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -232,7 +232,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/simplified.txt index 00cd0a509c..55c9b21954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (48) + WholeStageCodegen (30) Filter [web_cumulative,store_cumulative] HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] @@ -7,121 +7,107 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #8 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (21) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #11 - WholeStageCodegen (13) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (20) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + CometNativeColumnarToRow + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (7) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (6) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk,d_date] #5 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #8 + WholeStageCodegen (14) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [item_sk,d_date,sumss] #9 + WholeStageCodegen (13) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (9) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk,d_date] #11 + WholeStageCodegen (8) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (12) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter BroadcastExchange #13 - WholeStageCodegen (47) + WholeStageCodegen (29) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (46) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometNativeColumnarToRow + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt index 2cd07a9fb7..310de5b6a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometProject (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -436,7 +436,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt index f7d3371108..05c8752bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt @@ -33,7 +33,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -138,7 +138,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -161,7 +161,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt index b3013059b0..bcb2317588 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt @@ -43,12 +43,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt index 2cd07a9fb7..310de5b6a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt @@ -415,7 +415,7 @@ Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometProject (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -436,7 +436,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt index f7d3371108..05c8752bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt @@ -33,7 +33,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -138,7 +138,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -161,7 +161,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 b3013059b0..bcb2317588 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 @@ -43,12 +43,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/explain.txt index d2e466b254..49e8039970 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.call_center (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCode Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.catalog_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [2]: [cc_call_center_sk#14, cc_name#15] (17) BroadcastExchange Input [2]: [cc_call_center_sk#14, cc_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_call_center_sk#6] Right keys [1]: [cc_call_center_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] @@ -151,10 +151,10 @@ Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] (27) Window Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] (32) ReusedExchange [Reuses operator id: 21] Output [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] Keys [5]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(cs_sales_price#29))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#23, i_brand#24, cc_name#25, 5), ENSURE_RE Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, cc_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] (38) Window Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#23, i_brand#24, cc_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#23, i_brand#24, cc_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [5]: [i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] Input [7]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] @@ -228,13 +228,13 @@ Input [7]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sale Input [5]: [i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#23, i_brand#24, cc_name#25, (rn#31 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#30] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] @@ -245,14 +245,14 @@ Output [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sal Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] Arguments: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, cc_name#34 ASC NULLS FIRST, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] (46) Window Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] Arguments: [rank(d_year#35, d_moy#36) windowspecdefinition(i_category#32, i_brand#33, cc_name#34, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, cc_name#34], [d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [5]: [i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] Input [7]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37, rn#38] @@ -260,13 +260,13 @@ Input [7]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sale Input [5]: [i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#32, i_brand#33, cc_name#34, (rn#38 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#30 AS psum#39, sum_sales#37 AS nsum#40] Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#30, i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt index 65c66a7da8..66c510aaa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/simplified.txt index e603f05322..b61b5372fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [cs_item_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [cc_call_center_sk,cc_name] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt index 9265c65475..72c1da3c44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt index d655789fe6..d89e7759f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt index 9265c65475..72c1da3c44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 d655789fe6..d89e7759f3 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/explain.txt index 28d5613986..45cc6df163 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/explain.txt @@ -1,17 +1,17 @@ == Physical Plan == -* CometColumnarToRow (99) +CometNativeColumnarToRow (99) +- CometTakeOrderedAndProject (98) +- CometHashAggregate (97) +- CometColumnarExchange (96) +- * HashAggregate (95) +- Union (94) :- * HashAggregate (79) - : +- * CometColumnarToRow (78) + : +- CometNativeColumnarToRow (78) : +- CometColumnarExchange (77) : +- * HashAggregate (76) : +- Union (75) : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -29,12 +29,12 @@ : : : : +- Scan parquet spark_catalog.default.store_returns (5) : : : +- ReusedExchange (10) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometProject (15) : : +- CometFilter (14) : : +- CometNativeScan parquet spark_catalog.default.store (13) : :- * HashAggregate (46) - : : +- * CometColumnarToRow (45) + : : +- CometNativeColumnarToRow (45) : : +- CometColumnarExchange (44) : : +- * HashAggregate (43) : : +- * Project (42) @@ -52,12 +52,12 @@ : : : : +- Scan parquet spark_catalog.default.catalog_returns (28) : : : +- ReusedExchange (33) : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) + : : +- CometNativeColumnarToRow (39) : : +- CometProject (38) : : +- CometFilter (37) : : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) : +- * HashAggregate (74) - : +- * CometColumnarToRow (73) + : +- CometNativeColumnarToRow (73) : +- CometColumnarExchange (72) : +- * HashAggregate (71) : +- * Project (70) @@ -74,29 +74,29 @@ : : : :- BroadcastExchange (53) : : : : +- * ColumnarToRow (52) : : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : : +- * CometColumnarToRow (57) + : : : +- CometNativeColumnarToRow (57) : : : +- CometProject (56) : : : +- CometFilter (55) : : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) : : +- ReusedExchange (61) : +- BroadcastExchange (68) - : +- * CometColumnarToRow (67) + : +- CometNativeColumnarToRow (67) : +- CometProject (66) : +- CometFilter (65) : +- CometNativeScan parquet spark_catalog.default.web_site (64) :- * HashAggregate (86) - : +- * CometColumnarToRow (85) + : +- CometNativeColumnarToRow (85) : +- CometColumnarExchange (84) : +- * HashAggregate (83) : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) + : +- CometNativeColumnarToRow (81) : +- ReusedExchange (80) +- * HashAggregate (93) - +- * CometColumnarToRow (92) + +- CometNativeColumnarToRow (92) +- CometColumnarExchange (91) +- * HashAggregate (90) +- * HashAggregate (89) - +- * CometColumnarToRow (88) + +- CometNativeColumnarToRow (88) +- ReusedExchange (87) @@ -143,13 +143,13 @@ Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_s (10) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#22] -(11) BroadcastHashJoin [codegen id : 5] +(11) BroadcastHashJoin [codegen id : 3] Left keys [1]: [date_sk#7] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(12) Project [codegen id : 5] +(12) Project [codegen id : 3] Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] @@ -168,24 +168,24 @@ Condition : isnotnull(s_store_sk#23) Input [2]: [s_store_sk#23, s_store_id#24] Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#24, 16)) AS s_store_id#25] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#23, s_store_id#25] (17) BroadcastExchange Input [2]: [s_store_sk#23, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 5] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [store_sk#6] Right keys [1]: [s_store_sk#23] Join type: Inner Join condition: None -(19) Project [codegen id : 5] +(19) Project [codegen id : 3] Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] -(20) HashAggregate [codegen id : 5] +(20) HashAggregate [codegen id : 3] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] @@ -196,10 +196,10 @@ Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(23) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 4] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -214,14 +214,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] +(25) ColumnarToRow [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -(26) Filter [codegen id : 7] +(26) Filter [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Condition : isnotnull(cs_catalog_page_sk#43) -(27) Project [codegen id : 7] +(27) Project [codegen id : 5] Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] @@ -233,14 +233,14 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 8] +(29) ColumnarToRow [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -(30) Filter [codegen id : 8] +(30) Filter [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(31) Project [codegen id : 8] +(31) Project [codegen id : 6] Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] @@ -249,13 +249,13 @@ Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_retur (33) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#63] -(34) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(35) Project [codegen id : 11] +(35) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -274,24 +274,24 @@ Condition : isnotnull(cp_catalog_page_sk#64) Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#65, 16)) AS cp_catalog_page_id#66] -(39) CometColumnarToRow [codegen id : 10] +(39) CometNativeColumnarToRow Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] (40) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(41) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(42) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] @@ -302,10 +302,10 @@ Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(45) CometColumnarToRow [codegen id : 12] +(45) CometNativeColumnarToRow Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -(46) HashAggregate [codegen id : 12] +(46) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -320,14 +320,14 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 13] +(48) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(49) Filter [codegen id : 13] +(49) Filter [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Condition : isnotnull(ws_web_site_sk#84) -(50) Project [codegen id : 13] +(50) Project [codegen id : 9] Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] @@ -338,7 +338,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 14] +(52) ColumnarToRow [codegen id : 10] Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (53) BroadcastExchange @@ -360,16 +360,16 @@ Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND i Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(57) CometColumnarToRow +(57) CometNativeColumnarToRow Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(58) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 11] Left keys [2]: [wr_item_sk#94, wr_order_number#95] Right keys [2]: [ws_item_sk#99, ws_order_number#101] Join type: Inner Join condition: None -(59) Project [codegen id : 15] +(59) Project [codegen id : 11] Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] @@ -378,13 +378,13 @@ Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, (61) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#109] -(62) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 12] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(63) Project [codegen id : 18] +(63) Project [codegen id : 12] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -403,24 +403,24 @@ Condition : isnotnull(web_site_sk#110) Input [2]: [web_site_sk#110, web_site_id#111] Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#111, 16)) AS web_site_id#112] -(67) CometColumnarToRow [codegen id : 17] +(67) CometNativeColumnarToRow Input [2]: [web_site_sk#110, web_site_id#112] (68) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(69) BroadcastHashJoin [codegen id : 18] +(69) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(70) Project [codegen id : 18] +(70) Project [codegen id : 12] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] -(71) HashAggregate [codegen id : 18] +(71) HashAggregate [codegen id : 12] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Keys [1]: [web_site_id#112] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] @@ -431,10 +431,10 @@ Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -(74) HashAggregate [codegen id : 19] +(74) HashAggregate [codegen id : 13] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Keys [1]: [web_site_id#112] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] @@ -443,7 +443,7 @@ Results [5]: [web channel AS channel#125, concat(web_site, web_site_id#112) AS i (75) Union -(76) HashAggregate [codegen id : 20] +(76) HashAggregate [codegen id : 14] Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] Keys [2]: [channel#38, id#39] Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] @@ -454,10 +454,10 @@ Results [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(78) CometColumnarToRow [codegen id : 21] +(78) CometNativeColumnarToRow Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(79) HashAggregate [codegen id : 21] +(79) HashAggregate [codegen id : 15] Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Keys [2]: [channel#38, id#39] Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] @@ -467,17 +467,17 @@ Results [5]: [channel#38, id#39, cast(sum(sales#40)#142 as decimal(37,2)) AS sal (80) ReusedExchange [Reuses operator id: 77] Output [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -(81) CometColumnarToRow [codegen id : 42] +(81) CometNativeColumnarToRow Input [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -(82) HashAggregate [codegen id : 42] +(82) HashAggregate [codegen id : 30] Input [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] Keys [2]: [channel#148, id#149] Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] Aggregate Attributes [3]: [sum(sales#156)#142, sum(returns#157)#143, sum(profit#158)#144] Results [4]: [channel#148, sum(sales#156)#142 AS sales#159, sum(returns#157)#143 AS returns#160, sum(profit#158)#144 AS profit#161] -(83) HashAggregate [codegen id : 42] +(83) HashAggregate [codegen id : 30] Input [4]: [channel#148, sales#159, returns#160, profit#161] Keys [1]: [channel#148] Functions [3]: [partial_sum(sales#159), partial_sum(returns#160), partial_sum(profit#161)] @@ -488,10 +488,10 @@ Results [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] Arguments: hashpartitioning(channel#148, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(85) CometColumnarToRow [codegen id : 43] +(85) CometNativeColumnarToRow Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] -(86) HashAggregate [codegen id : 43] +(86) HashAggregate [codegen id : 31] Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] Keys [1]: [channel#148] Functions [3]: [sum(sales#159), sum(returns#160), sum(profit#161)] @@ -501,17 +501,17 @@ Results [5]: [channel#148, null AS id#177, sum(sales#159)#174 AS sum(sales)#178, (87) ReusedExchange [Reuses operator id: 77] Output [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -(88) CometColumnarToRow [codegen id : 64] +(88) CometNativeColumnarToRow Input [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -(89) HashAggregate [codegen id : 64] +(89) HashAggregate [codegen id : 46] Input [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] Keys [2]: [channel#181, id#182] Functions [3]: [sum(sales#189), sum(returns#190), sum(profit#191)] Aggregate Attributes [3]: [sum(sales#189)#142, sum(returns#190)#143, sum(profit#191)#144] Results [3]: [sum(sales#189)#142 AS sales#192, sum(returns#190)#143 AS returns#193, sum(profit#191)#144 AS profit#194] -(90) HashAggregate [codegen id : 64] +(90) HashAggregate [codegen id : 46] Input [3]: [sales#192, returns#193, profit#194] Keys: [] Functions [3]: [partial_sum(sales#192), partial_sum(returns#193), partial_sum(profit#194)] @@ -522,10 +522,10 @@ Results [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(92) CometColumnarToRow [codegen id : 65] +(92) CometNativeColumnarToRow Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -(93) HashAggregate [codegen id : 65] +(93) HashAggregate [codegen id : 47] Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] Functions [3]: [sum(sales#192), sum(returns#193), sum(profit#194)] @@ -534,7 +534,7 @@ Results [5]: [null AS channel#210, null AS id#211, sum(sales#192)#207 AS sum(sal (94) Union -(95) HashAggregate [codegen id : 66] +(95) HashAggregate [codegen id : 48] Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Functions: [] @@ -554,14 +554,14 @@ Functions: [] Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#145,returns#146,profit#147]), [channel#38, id#39, sales#145, returns#146, profit#147], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] -(99) CometColumnarToRow [codegen id : 67] +(99) CometNativeColumnarToRow Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (104) -+- * CometColumnarToRow (103) ++- CometNativeColumnarToRow (103) +- CometProject (102) +- CometFilter (101) +- CometNativeScan parquet spark_catalog.default.date_dim (100) @@ -582,7 +582,7 @@ Condition : (((isnotnull(d_date#215) AND (d_date#215 >= 1998-08-04)) AND (d_date Input [2]: [d_date_sk#22, d_date#215] Arguments: [d_date_sk#22], [d_date_sk#22] -(103) CometColumnarToRow [codegen id : 1] +(103) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (104) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt index 63ea8e5ea8..ec2eef025a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt @@ -1,16 +1,16 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,17 +34,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -63,17 +63,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -92,31 +92,31 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_site :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -130,7 +130,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -140,17 +140,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -169,17 +169,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -198,31 +198,31 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_site +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -236,7 +236,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -246,17 +246,17 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -275,17 +275,17 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.catalog_page +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -304,17 +304,17 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/simplified.txt index dc9c9b1ce7..ae0afc7f07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/simplified.txt @@ -1,167 +1,157 @@ -WholeStageCodegen (67) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (66) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (21) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (20) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (48) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (15) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (14) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_store_id] #3 + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #8 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 + Union + WholeStageCodegen (1) + Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] + Filter [ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + WholeStageCodegen (2) + Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] + Filter [sr_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (43) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #11 - WholeStageCodegen (42) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (65) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (64) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow + CometNativeColumnarToRow + CometColumnarExchange [cp_catalog_page_id] #6 + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (5) + Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] + Filter [cs_catalog_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + WholeStageCodegen (6) + Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] + Filter [cr_catalog_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (13) + HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometNativeColumnarToRow + CometColumnarExchange [web_site_id] #8 + WholeStageCodegen (12) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (9) + Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] + Filter [ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + WholeStageCodegen (11) + Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #10 + CometNativeColumnarToRow + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (31) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel] #11 + WholeStageCodegen (30) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (47) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #12 + WholeStageCodegen (46) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt index ca9ea23933..7c71867300 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt @@ -483,7 +483,7 @@ Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) @@ -504,7 +504,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt index 35b86dbff5..24167a2372 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,7 +197,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt index c0b236a57a..0084fda697 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt @@ -23,12 +23,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt index ca9ea23933..7c71867300 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt @@ -483,7 +483,7 @@ Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) @@ -504,7 +504,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt index 35b86dbff5..24167a2372 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,7 +197,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 c0b236a57a..0084fda697 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 @@ -23,12 +23,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/explain.txt index 8c97a06a14..2850db34b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (40) +- * Filter (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * Project (34) @@ -11,7 +11,7 @@ TakeOrderedAndProject (40) : +- * BroadcastHashJoin Inner BuildRight (17) : :- * Project (15) : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) + : : :- CometNativeColumnarToRow (9) : : : +- CometProject (8) : : : +- CometBroadcastHashJoin (7) : : : :- CometProject (3) @@ -26,7 +26,7 @@ TakeOrderedAndProject (40) : : +- Scan parquet spark_catalog.default.store_sales (10) : +- ReusedExchange (16) +- BroadcastExchange (32) - +- * CometColumnarToRow (31) + +- CometNativeColumnarToRow (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (20) @@ -80,7 +80,7 @@ Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(9) CometColumnarToRow [codegen id : 4] +(9) CometNativeColumnarToRow Input [2]: [ca_state#3, c_customer_sk#4] (10) Scan parquet spark_catalog.default.store_sales @@ -102,26 +102,26 @@ Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#4] Right keys [1]: [ss_customer_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] (16) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 2] Output [2]: [ca_state#3, ss_item_sk#6] Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] @@ -181,24 +181,24 @@ Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#13, Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] Arguments: [i_item_sk#11], [i_item_sk#11] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [i_item_sk#11] (32) BroadcastExchange Input [1]: [i_item_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 4] +(33) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#6] Right keys [1]: [i_item_sk#11] Join type: Inner Join condition: None -(34) Project [codegen id : 4] +(34) Project [codegen id : 2] Output [1]: [ca_state#3] Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] -(35) HashAggregate [codegen id : 4] +(35) HashAggregate [codegen id : 2] Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] @@ -209,17 +209,17 @@ Results [2]: [ca_state#3, count#21] Input [2]: [ca_state#3, count#21] Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometColumnarToRow [codegen id : 5] +(37) CometNativeColumnarToRow Input [2]: [ca_state#3, count#21] -(38) HashAggregate [codegen id : 5] +(38) HashAggregate [codegen id : 3] Input [2]: [ca_state#3, count#21] Keys [1]: [ca_state#3] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#22] Results [2]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24] -(39) Filter [codegen id : 5] +(39) Filter [codegen id : 3] Input [2]: [state#23, cnt#24] Condition : (cnt#24 >= 10) @@ -231,7 +231,7 @@ Arguments: 100, [cnt#24 ASC NULLS FIRST, state#23 ASC NULLS FIRST], [state#23, c Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = ReusedSubquery Sub Input [2]: [d_date_sk#10, d_month_seq#25] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -262,7 +262,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#26, [id=#27] Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -300,7 +300,7 @@ Input [1]: [d_month_seq#28] Keys [1]: [d_month_seq#28] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt index ee120ac920..44f11f06fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometProject @@ -25,13 +25,13 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -39,13 +39,13 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -53,7 +53,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/simplified.txt index 6b96848c69..95f9c38694 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) + WholeStageCodegen (3) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -13,8 +13,8 @@ TakeOrderedAndProject [cnt,state] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ca_state,ss_item_sk,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ca_state,c_customer_sk] CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] CometProject [ca_state] [ca_address_sk,ca_state] @@ -32,39 +32,33 @@ TakeOrderedAndProject [cnt,state] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_category,i_current_price] - CometFilter [i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #7 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #8 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_category,i_current_price] + CometFilter [i_current_price,i_category] + CometNativeScan 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/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/explain.txt index a41d75f6da..204a8c39c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#24, cnt#25] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Sub Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -262,7 +262,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -300,7 +300,7 @@ Input [1]: [d_month_seq#26] Keys [1]: [d_month_seq#26] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#26] Subquery:4 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt index 2d049f149f..4bb14d3c19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt @@ -23,13 +23,13 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/simplified.txt index 010720bc0e..76ccfa2abe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/simplified.txt @@ -25,23 +25,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt index a41d75f6da..204a8c39c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#24, cnt#25] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Sub Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -262,7 +262,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -300,7 +300,7 @@ Input [1]: [d_month_seq#26] Keys [1]: [d_month_seq#26] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#26] Subquery:4 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/extended.txt index 2d049f149f..4bb14d3c19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/extended.txt @@ -23,13 +23,13 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate 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 010720bc0e..76ccfa2abe 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 @@ -25,23 +25,19 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [d_month_seq] #2 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/explain.txt index 245bcb1bd2..c15c9e3e39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (180) +CometNativeColumnarToRow (180) +- CometSort (179) +- CometExchange (178) +- CometProject (177) @@ -48,7 +48,7 @@ : : : : : : : : : : : : : : : : : : +- * Filter (3) : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (8) : : : : : : : : : : : : : : : : : +- CometProject (7) : : : : : : : : : : : : : : : : : +- CometFilter (6) : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) @@ -153,7 +153,7 @@ : : : : : : : : : : : : : : : : : +- * Filter (108) : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (113) : : : : : : : : : : : : : : : : +- CometProject (112) : : : : : : : : : : : : : : : : +- CometFilter (111) : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) @@ -215,7 +215,7 @@ Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) CometColumnarToRow +(8) CometNativeColumnarToRow Input [2]: [sr_item_sk#14, sr_ticket_number#15] (9) BroadcastHashJoin [codegen id : 2] @@ -694,7 +694,7 @@ Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] -(113) CometColumnarToRow +(113) CometNativeColumnarToRow Input [2]: [sr_item_sk#111, sr_ticket_number#112] (114) BroadcastHashJoin [codegen id : 4] @@ -971,14 +971,14 @@ Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC 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#170, s2#171, s3#172, syear#168, cnt#169] 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#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 ASC NULLS FIRST] -(180) CometColumnarToRow [codegen id : 5] +(180) CometNativeColumnarToRow 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#170, s2#171, s3#172, syear#168, cnt#169] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometNativeScan parquet spark_catalog.default.date_dim (181) @@ -994,7 +994,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (184) BroadcastExchange @@ -1003,7 +1003,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (188) -+- * CometColumnarToRow (187) ++- CometNativeColumnarToRow (187) +- CometFilter (186) +- CometNativeScan parquet spark_catalog.default.date_dim (185) @@ -1019,7 +1019,7 @@ ReadSchema: struct Input [2]: [d_date_sk#124, d_year#125] Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(187) CometColumnarToRow [codegen id : 1] +(187) CometNativeColumnarToRow Input [2]: [d_date_sk#124, d_year#125] (188) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt index 6e3f37148b..98db64e5b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -49,10 +49,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -171,10 +171,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/simplified.txt index d933995ffc..c902a59f78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/simplified.txt @@ -1,206 +1,200 @@ -WholeStageCodegen (5) - 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] - 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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_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] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +CometNativeColumnarToRow + 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] + 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + WholeStageCodegen (2) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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_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] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometNativeScan 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + WholeStageCodegen (4) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #21 + WholeStageCodegen (3) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [d_date_sk,d_year] #12 ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt index 8764db8779..d2f47c05fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#124, d_year#125] Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#124, d_year#125] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt index 46fe063430..f1bd37b320 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] 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 8764db8779..d2f47c05fe 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 @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#124, d_year#125] Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#124, d_year#125] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject 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 46fe063430..f1bd37b320 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 @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/explain.txt index 5af8806175..f1dbe28e8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/explain.txt @@ -3,14 +3,14 @@ TakeOrderedAndProject (90) +- * Filter (89) +- Window (88) +- WindowGroupLimit (87) - +- * CometColumnarToRow (86) + +- CometNativeColumnarToRow (86) +- CometSort (85) +- CometColumnarExchange (84) +- WindowGroupLimit (83) +- * Sort (82) +- Union (81) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -24,70 +24,70 @@ TakeOrderedAndProject (90) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.item (14) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) + : +- CometNativeColumnarToRow (26) : +- ReusedExchange (25) :- * HashAggregate (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometColumnarExchange (36) : +- * HashAggregate (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) :- * HashAggregate (45) - : +- * CometColumnarToRow (44) + : +- CometNativeColumnarToRow (44) : +- CometColumnarExchange (43) : +- * HashAggregate (42) : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) + : +- CometNativeColumnarToRow (40) : +- ReusedExchange (39) :- * HashAggregate (52) - : +- * CometColumnarToRow (51) + : +- CometNativeColumnarToRow (51) : +- CometColumnarExchange (50) : +- * HashAggregate (49) : +- * HashAggregate (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- ReusedExchange (46) :- * HashAggregate (59) - : +- * CometColumnarToRow (58) + : +- CometNativeColumnarToRow (58) : +- CometColumnarExchange (57) : +- * HashAggregate (56) : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) + : +- CometNativeColumnarToRow (54) : +- ReusedExchange (53) :- * HashAggregate (66) - : +- * CometColumnarToRow (65) + : +- CometNativeColumnarToRow (65) : +- CometColumnarExchange (64) : +- * HashAggregate (63) : +- * HashAggregate (62) - : +- * CometColumnarToRow (61) + : +- CometNativeColumnarToRow (61) : +- ReusedExchange (60) :- * HashAggregate (73) - : +- * CometColumnarToRow (72) + : +- CometNativeColumnarToRow (72) : +- CometColumnarExchange (71) : +- * HashAggregate (70) : +- * HashAggregate (69) - : +- * CometColumnarToRow (68) + : +- CometNativeColumnarToRow (68) : +- ReusedExchange (67) +- * HashAggregate (80) - +- * CometColumnarToRow (79) + +- CometNativeColumnarToRow (79) +- CometColumnarExchange (78) +- * HashAggregate (77) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- ReusedExchange (74) @@ -99,23 +99,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 95] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] @@ -134,20 +134,20 @@ Condition : isnotnull(s_store_sk#11) Input [2]: [s_store_sk#11, s_store_id#12] Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#12, 16)) AS s_store_id#13] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [s_store_sk#11, s_store_id#13] (11) BroadcastExchange Input [2]: [s_store_sk#11, s_store_id#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] @@ -166,24 +166,24 @@ Condition : isnotnull(i_item_sk#14) Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#15, 50)) AS i_brand#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#16, 50)) AS i_class#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#17, 50)) AS i_category#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#18, 50)) AS i_product_name#22] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] (18) BroadcastExchange Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -194,10 +194,10 @@ Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year# Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 5] +(23) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -207,17 +207,17 @@ Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand# (25) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, s_store_id#44, sum#45, isEmpty#46] -(26) CometColumnarToRow [codegen id : 10] +(26) CometNativeColumnarToRow Input [10]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, s_store_id#44, sum#45, isEmpty#46] -(27) HashAggregate [codegen id : 10] +(27) HashAggregate [codegen id : 4] Input [10]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, s_store_id#44, sum#45, isEmpty#46] Keys [8]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, s_store_id#44] Functions [1]: [sum(coalesce((ss_sales_price#47 * cast(ss_quantity#48 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#47 * cast(ss_quantity#48 as decimal(10,0))), 0.00))#27] Results [8]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sum(coalesce((ss_sales_price#47 * cast(ss_quantity#48 as decimal(10,0))), 0.00))#27 AS sumsales#49] -(28) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 4] Input [8]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sumsales#49] Keys [7]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43] Functions [1]: [partial_sum(sumsales#49)] @@ -228,10 +228,10 @@ Results [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#4 Input [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sum#52, isEmpty#53] Arguments: hashpartitioning(i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 11] +(30) CometNativeColumnarToRow Input [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sum#52, isEmpty#53] -(31) HashAggregate [codegen id : 11] +(31) HashAggregate [codegen id : 5] Input [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sum#52, isEmpty#53] Keys [7]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43] Functions [1]: [sum(sumsales#49)] @@ -241,17 +241,17 @@ Results [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#4 (32) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, d_moy#63, s_store_id#64, sum#65, isEmpty#66] -(33) CometColumnarToRow [codegen id : 16] +(33) CometNativeColumnarToRow Input [10]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, d_moy#63, s_store_id#64, sum#65, isEmpty#66] -(34) HashAggregate [codegen id : 16] +(34) HashAggregate [codegen id : 7] Input [10]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, d_moy#63, s_store_id#64, sum#65, isEmpty#66] Keys [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, d_moy#63, s_store_id#64] Functions [1]: [sum(coalesce((ss_sales_price#67 * cast(ss_quantity#68 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#67 * cast(ss_quantity#68 as decimal(10,0))), 0.00))#27] Results [7]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sum(coalesce((ss_sales_price#67 * cast(ss_quantity#68 as decimal(10,0))), 0.00))#27 AS sumsales#69] -(35) HashAggregate [codegen id : 16] +(35) HashAggregate [codegen id : 7] Input [7]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sumsales#69] Keys [6]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62] Functions [1]: [partial_sum(sumsales#69)] @@ -262,10 +262,10 @@ Results [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#6 Input [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sum#72, isEmpty#73] Arguments: hashpartitioning(i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 17] +(37) CometNativeColumnarToRow Input [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sum#72, isEmpty#73] -(38) HashAggregate [codegen id : 17] +(38) HashAggregate [codegen id : 8] Input [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sum#72, isEmpty#73] Keys [6]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62] Functions [1]: [sum(sumsales#69)] @@ -275,17 +275,17 @@ Results [9]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#6 (39) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, d_qoy#83, d_moy#84, s_store_id#85, sum#86, isEmpty#87] -(40) CometColumnarToRow [codegen id : 22] +(40) CometNativeColumnarToRow Input [10]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, d_qoy#83, d_moy#84, s_store_id#85, sum#86, isEmpty#87] -(41) HashAggregate [codegen id : 22] +(41) HashAggregate [codegen id : 10] Input [10]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, d_qoy#83, d_moy#84, s_store_id#85, sum#86, isEmpty#87] Keys [8]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, d_qoy#83, d_moy#84, s_store_id#85] Functions [1]: [sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))#27] Results [6]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))#27 AS sumsales#90] -(42) HashAggregate [codegen id : 22] +(42) HashAggregate [codegen id : 10] Input [6]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sumsales#90] Keys [5]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82] Functions [1]: [partial_sum(sumsales#90)] @@ -296,10 +296,10 @@ Results [7]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#8 Input [7]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sum#93, isEmpty#94] Arguments: hashpartitioning(i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(44) CometColumnarToRow [codegen id : 23] +(44) CometNativeColumnarToRow Input [7]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sum#93, isEmpty#94] -(45) HashAggregate [codegen id : 23] +(45) HashAggregate [codegen id : 11] Input [7]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sum#93, isEmpty#94] Keys [5]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82] Functions [1]: [sum(sumsales#90)] @@ -309,17 +309,17 @@ Results [9]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#8 (46) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, d_year#104, d_qoy#105, d_moy#106, s_store_id#107, sum#108, isEmpty#109] -(47) CometColumnarToRow [codegen id : 28] +(47) CometNativeColumnarToRow Input [10]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, d_year#104, d_qoy#105, d_moy#106, s_store_id#107, sum#108, isEmpty#109] -(48) HashAggregate [codegen id : 28] +(48) HashAggregate [codegen id : 13] Input [10]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, d_year#104, d_qoy#105, d_moy#106, s_store_id#107, sum#108, isEmpty#109] Keys [8]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, d_year#104, d_qoy#105, d_moy#106, s_store_id#107] Functions [1]: [sum(coalesce((ss_sales_price#110 * cast(ss_quantity#111 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#110 * cast(ss_quantity#111 as decimal(10,0))), 0.00))#27] Results [5]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum(coalesce((ss_sales_price#110 * cast(ss_quantity#111 as decimal(10,0))), 0.00))#27 AS sumsales#112] -(49) HashAggregate [codegen id : 28] +(49) HashAggregate [codegen id : 13] Input [5]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sumsales#112] Keys [4]: [i_category#100, i_class#101, i_brand#102, i_product_name#103] Functions [1]: [partial_sum(sumsales#112)] @@ -330,10 +330,10 @@ Results [6]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum# Input [6]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum#115, isEmpty#116] Arguments: hashpartitioning(i_category#100, i_class#101, i_brand#102, i_product_name#103, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) CometColumnarToRow [codegen id : 29] +(51) CometNativeColumnarToRow Input [6]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum#115, isEmpty#116] -(52) HashAggregate [codegen id : 29] +(52) HashAggregate [codegen id : 14] Input [6]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum#115, isEmpty#116] Keys [4]: [i_category#100, i_class#101, i_brand#102, i_product_name#103] Functions [1]: [sum(sumsales#112)] @@ -343,17 +343,17 @@ Results [9]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, null (53) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#123, i_class#124, i_brand#125, i_product_name#126, d_year#127, d_qoy#128, d_moy#129, s_store_id#130, sum#131, isEmpty#132] -(54) CometColumnarToRow [codegen id : 34] +(54) CometNativeColumnarToRow Input [10]: [i_category#123, i_class#124, i_brand#125, i_product_name#126, d_year#127, d_qoy#128, d_moy#129, s_store_id#130, sum#131, isEmpty#132] -(55) HashAggregate [codegen id : 34] +(55) HashAggregate [codegen id : 16] Input [10]: [i_category#123, i_class#124, i_brand#125, i_product_name#126, d_year#127, d_qoy#128, d_moy#129, s_store_id#130, sum#131, isEmpty#132] Keys [8]: [i_category#123, i_class#124, i_brand#125, i_product_name#126, d_year#127, d_qoy#128, d_moy#129, s_store_id#130] Functions [1]: [sum(coalesce((ss_sales_price#133 * cast(ss_quantity#134 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#133 * cast(ss_quantity#134 as decimal(10,0))), 0.00))#27] Results [4]: [i_category#123, i_class#124, i_brand#125, sum(coalesce((ss_sales_price#133 * cast(ss_quantity#134 as decimal(10,0))), 0.00))#27 AS sumsales#135] -(56) HashAggregate [codegen id : 34] +(56) HashAggregate [codegen id : 16] Input [4]: [i_category#123, i_class#124, i_brand#125, sumsales#135] Keys [3]: [i_category#123, i_class#124, i_brand#125] Functions [1]: [partial_sum(sumsales#135)] @@ -364,10 +364,10 @@ Results [5]: [i_category#123, i_class#124, i_brand#125, sum#138, isEmpty#139] Input [5]: [i_category#123, i_class#124, i_brand#125, sum#138, isEmpty#139] Arguments: hashpartitioning(i_category#123, i_class#124, i_brand#125, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(58) CometColumnarToRow [codegen id : 35] +(58) CometNativeColumnarToRow Input [5]: [i_category#123, i_class#124, i_brand#125, sum#138, isEmpty#139] -(59) HashAggregate [codegen id : 35] +(59) HashAggregate [codegen id : 17] Input [5]: [i_category#123, i_class#124, i_brand#125, sum#138, isEmpty#139] Keys [3]: [i_category#123, i_class#124, i_brand#125] Functions [1]: [sum(sumsales#135)] @@ -377,17 +377,17 @@ Results [9]: [i_category#123, i_class#124, i_brand#125, null AS i_product_name#1 (60) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#147, i_class#148, i_brand#149, i_product_name#150, d_year#151, d_qoy#152, d_moy#153, s_store_id#154, sum#155, isEmpty#156] -(61) CometColumnarToRow [codegen id : 40] +(61) CometNativeColumnarToRow Input [10]: [i_category#147, i_class#148, i_brand#149, i_product_name#150, d_year#151, d_qoy#152, d_moy#153, s_store_id#154, sum#155, isEmpty#156] -(62) HashAggregate [codegen id : 40] +(62) HashAggregate [codegen id : 19] Input [10]: [i_category#147, i_class#148, i_brand#149, i_product_name#150, d_year#151, d_qoy#152, d_moy#153, s_store_id#154, sum#155, isEmpty#156] Keys [8]: [i_category#147, i_class#148, i_brand#149, i_product_name#150, d_year#151, d_qoy#152, d_moy#153, s_store_id#154] Functions [1]: [sum(coalesce((ss_sales_price#157 * cast(ss_quantity#158 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#157 * cast(ss_quantity#158 as decimal(10,0))), 0.00))#27] Results [3]: [i_category#147, i_class#148, sum(coalesce((ss_sales_price#157 * cast(ss_quantity#158 as decimal(10,0))), 0.00))#27 AS sumsales#159] -(63) HashAggregate [codegen id : 40] +(63) HashAggregate [codegen id : 19] Input [3]: [i_category#147, i_class#148, sumsales#159] Keys [2]: [i_category#147, i_class#148] Functions [1]: [partial_sum(sumsales#159)] @@ -398,10 +398,10 @@ Results [4]: [i_category#147, i_class#148, sum#162, isEmpty#163] Input [4]: [i_category#147, i_class#148, sum#162, isEmpty#163] Arguments: hashpartitioning(i_category#147, i_class#148, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(65) CometColumnarToRow [codegen id : 41] +(65) CometNativeColumnarToRow Input [4]: [i_category#147, i_class#148, sum#162, isEmpty#163] -(66) HashAggregate [codegen id : 41] +(66) HashAggregate [codegen id : 20] Input [4]: [i_category#147, i_class#148, sum#162, isEmpty#163] Keys [2]: [i_category#147, i_class#148] Functions [1]: [sum(sumsales#159)] @@ -411,17 +411,17 @@ Results [9]: [i_category#147, i_class#148, null AS i_brand#165, null AS i_produc (67) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] -(68) CometColumnarToRow [codegen id : 46] +(68) CometNativeColumnarToRow Input [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] -(69) HashAggregate [codegen id : 46] +(69) HashAggregate [codegen id : 22] Input [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] Keys [8]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179] Functions [1]: [sum(coalesce((ss_sales_price#182 * cast(ss_quantity#183 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#182 * cast(ss_quantity#183 as decimal(10,0))), 0.00))#27] Results [2]: [i_category#172, sum(coalesce((ss_sales_price#182 * cast(ss_quantity#183 as decimal(10,0))), 0.00))#27 AS sumsales#184] -(70) HashAggregate [codegen id : 46] +(70) HashAggregate [codegen id : 22] Input [2]: [i_category#172, sumsales#184] Keys [1]: [i_category#172] Functions [1]: [partial_sum(sumsales#184)] @@ -432,10 +432,10 @@ Results [3]: [i_category#172, sum#187, isEmpty#188] Input [3]: [i_category#172, sum#187, isEmpty#188] Arguments: hashpartitioning(i_category#172, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(72) CometColumnarToRow [codegen id : 47] +(72) CometNativeColumnarToRow Input [3]: [i_category#172, sum#187, isEmpty#188] -(73) HashAggregate [codegen id : 47] +(73) HashAggregate [codegen id : 23] Input [3]: [i_category#172, sum#187, isEmpty#188] Keys [1]: [i_category#172] Functions [1]: [sum(sumsales#184)] @@ -445,17 +445,17 @@ Results [9]: [i_category#172, null AS i_class#190, null AS i_brand#191, null AS (74) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] -(75) CometColumnarToRow [codegen id : 52] +(75) CometNativeColumnarToRow Input [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] -(76) HashAggregate [codegen id : 52] +(76) HashAggregate [codegen id : 25] Input [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] Keys [8]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205] Functions [1]: [sum(coalesce((ss_sales_price#208 * cast(ss_quantity#209 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#208 * cast(ss_quantity#209 as decimal(10,0))), 0.00))#27] Results [1]: [sum(coalesce((ss_sales_price#208 * cast(ss_quantity#209 as decimal(10,0))), 0.00))#27 AS sumsales#210] -(77) HashAggregate [codegen id : 52] +(77) HashAggregate [codegen id : 25] Input [1]: [sumsales#210] Keys: [] Functions [1]: [partial_sum(sumsales#210)] @@ -466,10 +466,10 @@ Results [2]: [sum#213, isEmpty#214] Input [2]: [sum#213, isEmpty#214] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(79) CometColumnarToRow [codegen id : 53] +(79) CometNativeColumnarToRow Input [2]: [sum#213, isEmpty#214] -(80) HashAggregate [codegen id : 53] +(80) HashAggregate [codegen id : 26] Input [2]: [sum#213, isEmpty#214] Keys: [] Functions [1]: [sum(sumsales#210)] @@ -478,7 +478,7 @@ Results [9]: [null AS i_category#216, null AS i_class#217, null AS i_brand#218, (81) Union -(82) Sort [codegen id : 54] +(82) Sort [codegen id : 27] Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] Arguments: [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST], false, 0 @@ -494,7 +494,7 @@ Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumna Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] -(86) CometColumnarToRow [codegen id : 55] +(86) CometNativeColumnarToRow Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] (87) WindowGroupLimit @@ -505,7 +505,7 @@ Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 10 Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#225], [i_category#28], [sumsales#36 DESC NULLS LAST] -(89) Filter [codegen id : 56] +(89) Filter [codegen id : 28] Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#225] Condition : (rk#225 <= 100) @@ -517,7 +517,7 @@ Arguments: 100, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (95) -+- * CometColumnarToRow (94) ++- CometNativeColumnarToRow (94) +- CometProject (93) +- CometFilter (92) +- CometNativeScan parquet spark_catalog.default.date_dim (91) @@ -538,7 +538,7 @@ Condition : (((isnotnull(d_month_seq#226) AND (d_month_seq#226 >= 1212)) AND (d_ Input [5]: [d_date_sk#7, d_month_seq#226, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(94) CometColumnarToRow [codegen id : 1] +(94) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (95) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt index 0d338d5831..bf553c1971 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt @@ -2,14 +2,14 @@ TakeOrderedAndProject +- Filter +- Window +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- WindowGroupLimit +- Sort +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -23,31 +23,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -61,31 +61,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -99,31 +99,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -137,31 +137,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -175,31 +175,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -213,31 +213,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -251,31 +251,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -289,31 +289,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -327,22 +327,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/simplified.txt index 38fa0be072..bdb19b6fb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/simplified.txt @@ -1,147 +1,139 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (56) + WholeStageCodegen (28) Filter [rk] InputAdapter Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (55) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (54) - Sort [i_category,sumsales] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (27) + Sort [i_category,sumsales] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #11 - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #12 - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (8) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (11) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 + WholeStageCodegen (10) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (14) + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 + WholeStageCodegen (13) + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand] #10 + WholeStageCodegen (16) + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (20) + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class] #11 + WholeStageCodegen (19) + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (23) + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category] #12 + WholeStageCodegen (22) + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (26) + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #13 + WholeStageCodegen (25) + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + 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/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/explain.txt index 481b95c617..fca73f7826 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/explain.txt @@ -414,7 +414,7 @@ Arguments: 100, [i_category#146 ASC NULLS FIRST, i_class#147 ASC NULLS FIRST, i_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometFilter (76) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) @@ -435,7 +435,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt index b96ce25ae1..407fc9836c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -54,7 +54,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -86,7 +86,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -118,7 +118,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -150,7 +150,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -182,7 +182,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -214,7 +214,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -246,7 +246,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -278,7 +278,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/simplified.txt index 6096ee1c25..59865ee9ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/simplified.txt @@ -28,12 +28,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt index 481b95c617..fca73f7826 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt @@ -414,7 +414,7 @@ Arguments: 100, [i_category#146 ASC NULLS FIRST, i_class#147 ASC NULLS FIRST, i_ Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (79) -+- * CometColumnarToRow (78) ++- CometNativeColumnarToRow (78) +- CometProject (77) +- CometFilter (76) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) @@ -435,7 +435,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(78) CometColumnarToRow [codegen id : 1] +(78) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (79) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/extended.txt index b96ce25ae1..407fc9836c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -54,7 +54,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -86,7 +86,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -118,7 +118,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -150,7 +150,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -182,7 +182,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -214,7 +214,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -246,7 +246,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -278,7 +278,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 6096ee1c25..59865ee9ad 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 @@ -28,12 +28,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/explain.txt index b203fef099..06b8f24849 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (65) +- * Project (64) +- Window (63) - +- * CometColumnarToRow (62) + +- CometNativeColumnarToRow (62) +- CometSort (61) +- CometExchange (60) +- CometHashAggregate (59) @@ -10,7 +10,7 @@ TakeOrderedAndProject (65) +- * HashAggregate (57) +- Union (56) :- * HashAggregate (41) - : +- * CometColumnarToRow (40) + : +- CometNativeColumnarToRow (40) : +- CometColumnarExchange (39) : +- * HashAggregate (38) : +- * Project (37) @@ -24,7 +24,7 @@ TakeOrderedAndProject (65) : +- BroadcastExchange (35) : +- * Project (34) : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (9) + : :- CometNativeColumnarToRow (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (32) @@ -34,7 +34,7 @@ TakeOrderedAndProject (65) : +- WindowGroupLimit (28) : +- * Sort (27) : +- * HashAggregate (26) - : +- * CometColumnarToRow (25) + : +- CometNativeColumnarToRow (25) : +- CometColumnarExchange (24) : +- * HashAggregate (23) : +- * Project (22) @@ -45,24 +45,24 @@ TakeOrderedAndProject (65) : : : +- * ColumnarToRow (11) : : : +- Scan parquet spark_catalog.default.store_sales (10) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometProject (15) : : +- CometFilter (14) : : +- CometNativeScan parquet spark_catalog.default.store (13) : +- ReusedExchange (20) :- * HashAggregate (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometColumnarExchange (46) : +- * HashAggregate (45) : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- ReusedExchange (42) +- * HashAggregate (55) - +- * CometColumnarToRow (54) + +- CometNativeColumnarToRow (54) +- CometColumnarExchange (53) +- * HashAggregate (52) +- * HashAggregate (51) - +- * CometColumnarToRow (50) + +- CometNativeColumnarToRow (50) +- ReusedExchange (49) @@ -74,23 +74,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 70] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 5] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -105,7 +105,7 @@ ReadSchema: struct Input [3]: [s_store_sk#6, s_county#7, s_state#8] Condition : isnotnull(s_store_sk#6) -(9) CometColumnarToRow [codegen id : 7] +(9) CometNativeColumnarToRow Input [3]: [s_store_sk#6, s_county#7, s_state#8] (10) Scan parquet spark_catalog.default.store_sales @@ -116,10 +116,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Condition : isnotnull(ss_store_sk#9) @@ -138,37 +138,37 @@ Condition : isnotnull(s_store_sk#12) Input [2]: [s_store_sk#12, s_state#13] Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#14] -(16) CometColumnarToRow [codegen id : 2] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#12, s_state#14] (17) BroadcastExchange Input [2]: [s_store_sk#12, s_state#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#9] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] (20) ReusedExchange [Reuses operator id: 70] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 4] +(21) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#11] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 4] +(22) Project [codegen id : 1] Output [2]: [ss_net_profit#10, s_state#14] Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] -(23) HashAggregate [codegen id : 4] +(23) HashAggregate [codegen id : 1] Input [2]: [ss_net_profit#10, s_state#14] Keys [1]: [s_state#14] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] @@ -179,17 +179,17 @@ Results [2]: [s_state#14, sum#17] Input [2]: [s_state#14, sum#17] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(25) CometColumnarToRow [codegen id : 5] +(25) CometNativeColumnarToRow Input [2]: [s_state#14, sum#17] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 2] Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(27) Sort [codegen id : 5] +(27) Sort [codegen id : 2] Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 @@ -201,11 +201,11 @@ Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] -(30) Filter [codegen id : 6] +(30) Filter [codegen id : 3] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Condition : (ranking#20 <= 5) -(31) Project [codegen id : 6] +(31) Project [codegen id : 3] Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] @@ -213,13 +213,13 @@ Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 7] +(33) BroadcastHashJoin [codegen id : 4] Left keys [1]: [static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#8, 2))] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(34) Project [codegen id : 7] +(34) Project [codegen id : 4] Output [3]: [s_store_sk#6, s_county#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#8, 2)) AS s_state#21] Input [3]: [s_store_sk#6, s_county#7, s_state#8] @@ -227,17 +227,17 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] Input [3]: [s_store_sk#6, s_county#7, s_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(36) BroadcastHashJoin [codegen id : 8] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(37) Project [codegen id : 8] +(37) Project [codegen id : 5] Output [3]: [ss_net_profit#2, s_county#7, s_state#21] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] -(38) HashAggregate [codegen id : 8] +(38) HashAggregate [codegen id : 5] Input [3]: [ss_net_profit#2, s_county#7, s_state#21] Keys [2]: [s_state#21, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] @@ -248,10 +248,10 @@ Results [3]: [s_state#21, s_county#7, sum#23] Input [3]: [s_state#21, s_county#7, sum#23] Arguments: hashpartitioning(s_state#21, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometColumnarToRow [codegen id : 9] +(40) CometNativeColumnarToRow Input [3]: [s_state#21, s_county#7, sum#23] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 6] Input [3]: [s_state#21, s_county#7, sum#23] Keys [2]: [s_state#21, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] @@ -261,17 +261,17 @@ Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as d (42) ReusedExchange [Reuses operator id: 39] Output [3]: [s_state#31, s_county#32, sum#33] -(43) CometColumnarToRow [codegen id : 18] +(43) CometNativeColumnarToRow Input [3]: [s_state#31, s_county#32, sum#33] -(44) HashAggregate [codegen id : 18] +(44) HashAggregate [codegen id : 12] Input [3]: [s_state#31, s_county#32, sum#33] Keys [2]: [s_state#31, s_county#32] Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#24] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#24,17,2) AS total_sum#35, s_state#31] -(45) HashAggregate [codegen id : 18] +(45) HashAggregate [codegen id : 12] Input [2]: [total_sum#35, s_state#31] Keys [1]: [s_state#31] Functions [1]: [partial_sum(total_sum#35)] @@ -282,10 +282,10 @@ Results [3]: [s_state#31, sum#38, isEmpty#39] Input [3]: [s_state#31, sum#38, isEmpty#39] Arguments: hashpartitioning(s_state#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(47) CometColumnarToRow [codegen id : 19] +(47) CometNativeColumnarToRow Input [3]: [s_state#31, sum#38, isEmpty#39] -(48) HashAggregate [codegen id : 19] +(48) HashAggregate [codegen id : 13] Input [3]: [s_state#31, sum#38, isEmpty#39] Keys [1]: [s_state#31] Functions [1]: [sum(total_sum#35)] @@ -295,17 +295,17 @@ Results [6]: [sum(total_sum#35)#40 AS total_sum#41, s_state#31, null AS s_county (49) ReusedExchange [Reuses operator id: 39] Output [3]: [s_state#46, s_county#47, sum#48] -(50) CometColumnarToRow [codegen id : 28] +(50) CometNativeColumnarToRow Input [3]: [s_state#46, s_county#47, sum#48] -(51) HashAggregate [codegen id : 28] +(51) HashAggregate [codegen id : 19] Input [3]: [s_state#46, s_county#47, sum#48] Keys [2]: [s_state#46, s_county#47] Functions [1]: [sum(UnscaledValue(ss_net_profit#49))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#49))#24] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#49))#24,17,2) AS total_sum#50] -(52) HashAggregate [codegen id : 28] +(52) HashAggregate [codegen id : 19] Input [1]: [total_sum#50] Keys: [] Functions [1]: [partial_sum(total_sum#50)] @@ -316,10 +316,10 @@ Results [2]: [sum#53, isEmpty#54] Input [2]: [sum#53, isEmpty#54] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(54) CometColumnarToRow [codegen id : 29] +(54) CometNativeColumnarToRow Input [2]: [sum#53, isEmpty#54] -(55) HashAggregate [codegen id : 29] +(55) HashAggregate [codegen id : 20] Input [2]: [sum#53, isEmpty#54] Keys: [] Functions [1]: [sum(total_sum#50)] @@ -328,7 +328,7 @@ Results [6]: [sum(total_sum#50)#55 AS total_sum#56, null AS s_state#57, null AS (56) Union -(57) HashAggregate [codegen id : 30] +(57) HashAggregate [codegen id : 21] Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] @@ -352,14 +352,14 @@ Arguments: hashpartitioning(lochierarchy#30, _w0#62, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62], [lochierarchy#30 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(62) CometColumnarToRow [codegen id : 31] +(62) CometNativeColumnarToRow Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] (63) Window Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#62, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#30, _w0#62], [total_sum#25 DESC NULLS LAST] -(64) Project [codegen id : 32] +(64) Project [codegen id : 22] Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62, rank_within_parent#63] @@ -371,7 +371,7 @@ Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometNativeScan parquet spark_catalog.default.date_dim (66) @@ -392,7 +392,7 @@ Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#64] Arguments: [d_date_sk#5], [d_date_sk#5] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt index d13ccdd19f..3214d035b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -9,7 +9,7 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,19 +21,19 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange @@ -43,7 +43,7 @@ TakeOrderedAndProject : +- WindowGroupLimit : +- Sort : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -55,21 +55,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -81,19 +81,19 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange @@ -103,7 +103,7 @@ TakeOrderedAndProject : +- WindowGroupLimit : +- Sort : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -115,21 +115,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -141,19 +141,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange @@ -163,7 +163,7 @@ TakeOrderedAndProject +- WindowGroupLimit +- Sort +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -175,12 +175,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/simplified.txt index ba40c1d591..fb6fc1844f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/simplified.txt @@ -1,107 +1,101 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (32) + WholeStageCodegen (22) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (31) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - 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 (30) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (9) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + 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 (21) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + InputAdapter + CometNativeColumnarToRow + CometFilter [s_store_sk,s_county,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (2) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state] #7 + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #9 - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (13) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state] #9 + WholeStageCodegen (12) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (20) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (19) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt index a66acf75ca..37443b3154 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject (66) : +- BroadcastExchange (36) : +- * Project (35) : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) + : :- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) : +- BroadcastExchange (33) @@ -121,7 +121,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -375,7 +375,7 @@ Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) @@ -396,7 +396,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt index 191d0ef18a..0ab48d6a12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -77,7 +77,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -88,7 +88,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -133,7 +133,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -144,7 +144,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt index 2832c5a348..69d6149130 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -46,8 +44,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt index a66acf75ca..37443b3154 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject (66) : +- BroadcastExchange (36) : +- * Project (35) : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) + : :- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) : +- BroadcastExchange (33) @@ -121,7 +121,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -375,7 +375,7 @@ Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometProject (69) +- CometFilter (68) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) @@ -396,7 +396,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt index 191d0ef18a..0ab48d6a12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -77,7 +77,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -88,7 +88,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -133,7 +133,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -144,7 +144,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange 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 2832c5a348..69d6149130 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 @@ -31,12 +31,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -46,8 +44,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/explain.txt index 24b9e03cc2..c91c529620 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (70) +CometNativeColumnarToRow (70) +- CometTakeOrderedAndProject (69) +- CometHashAggregate (68) +- CometExchange (67) @@ -30,38 +30,38 @@ : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) + : : : : : : : : : +- CometNativeColumnarToRow (6) : : : : : : : : : +- CometFilter (5) : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) + : : : : : : : : +- CometNativeColumnarToRow (12) : : : : : : : : +- CometFilter (11) : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometNativeColumnarToRow (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) + : : : : : : +- CometNativeColumnarToRow (25) : : : : : : +- CometProject (24) : : : : : : +- CometFilter (23) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) + : : : : : +- CometNativeColumnarToRow (32) : : : : : +- CometProject (31) : : : : : +- CometFilter (30) : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) : : : : +- ReusedExchange (36) : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) + : : : +- CometNativeColumnarToRow (41) : : : +- CometFilter (40) : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) + : : +- CometNativeColumnarToRow (47) : : +- CometFilter (46) : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometFilter (52) : +- CometNativeScan parquet spark_catalog.default.promotion (51) +- CometSort (63) @@ -79,10 +79,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 10] +(2) ColumnarToRow [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -(3) Filter [codegen id : 10] +(3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) @@ -98,20 +98,20 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] (13) BroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 10] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#11] Right keys [1]: [w_warehouse_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 10] +(15) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] @@ -154,20 +154,20 @@ ReadSchema: struct Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) CometColumnarToRow [codegen id : 3] +(18) CometNativeColumnarToRow Input [2]: [i_item_sk#16, i_item_desc#17] (19) BroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 10] +(20) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#4] Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(21) Project [codegen id : 10] +(21) Project [codegen id : 1] Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] @@ -186,20 +186,20 @@ Condition : ((isnotnull(cd_marital_status#19) AND (static_invoke(CharVarcharCode Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) CometColumnarToRow [codegen id : 4] +(25) CometNativeColumnarToRow Input [1]: [cd_demo_sk#18] (26) BroadcastExchange Input [1]: [cd_demo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 10] +(27) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(28) Project [codegen id : 10] +(28) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] @@ -218,33 +218,33 @@ Condition : ((isnotnull(hd_buy_potential#21) AND (static_invoke(CharVarcharCodeg Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) CometColumnarToRow [codegen id : 5] +(32) CometNativeColumnarToRow Input [1]: [hd_demo_sk#20] (33) BroadcastExchange Input [1]: [hd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_hdemo_sk#3] Right keys [1]: [hd_demo_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 10] +(35) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] (36) ReusedExchange [Reuses operator id: 75] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(37) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#8] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(38) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] @@ -259,20 +259,20 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_week_seq#26] Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) -(41) CometColumnarToRow [codegen id : 7] +(41) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_week_seq#26] (42) BroadcastExchange Input [2]: [d_date_sk#25, d_week_seq#26] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 10] +(43) BroadcastHashJoin [codegen id : 1] Left keys [2]: [d_week_seq#24, inv_date_sk#13] Right keys [2]: [d_week_seq#26, d_date_sk#25] Join type: Inner Join condition: None -(44) Project [codegen id : 10] +(44) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] @@ -287,20 +287,20 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_date#28] Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) -(47) CometColumnarToRow [codegen id : 8] +(47) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_date#28] (48) BroadcastExchange Input [2]: [d_date_sk#27, d_date#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: (d_date#28 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(50) Project [codegen id : 1] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] @@ -315,20 +315,20 @@ ReadSchema: struct Input [1]: [p_promo_sk#29] Condition : isnotnull(p_promo_sk#29) -(53) CometColumnarToRow [codegen id : 9] +(53) CometNativeColumnarToRow Input [1]: [p_promo_sk#29] (54) BroadcastExchange Input [1]: [p_promo_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(55) BroadcastHashJoin [codegen id : 10] +(55) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_promo_sk#5] Right keys [1]: [p_promo_sk#29] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(56) Project [codegen id : 1] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] @@ -390,14 +390,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -(70) CometColumnarToRow [codegen id : 11] +(70) CometNativeColumnarToRow Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (75) -+- * CometColumnarToRow (74) ++- CometNativeColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -418,7 +418,7 @@ Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) CometColumnarToRow [codegen id : 1] +(74) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt index 081972705a..62c2b17b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -30,47 +30,47 @@ CometColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/simplified.txt index 025217a607..edcf2b00f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/simplified.txt @@ -1,107 +1,87 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - 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] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + 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] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + BroadcastExchange #6 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + BroadcastExchange #9 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + BroadcastExchange #10 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + CometNativeColumnarToRow + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/explain.txt index 707d448aff..81c526625b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/simplified.txt index 678f8c67df..549469c03b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt index 707d448aff..81c526625b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 678f8c67df..549469c03b 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 @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/explain.txt index 0ac606319d..e631a4d566 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/explain.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject (79) : :- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -25,14 +25,14 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -44,14 +44,14 @@ TakeOrderedAndProject (79) : +- BroadcastExchange (56) : +- * Filter (55) : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (47) : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) + : : :- CometNativeColumnarToRow (41) : : : +- CometProject (40) : : : +- CometFilter (39) : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -62,14 +62,14 @@ TakeOrderedAndProject (79) : +- ReusedExchange (48) +- BroadcastExchange (76) +- * HashAggregate (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometColumnarExchange (73) +- * HashAggregate (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) :- * Project (68) : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) + : :- CometNativeColumnarToRow (62) : : +- CometProject (61) : : +- CometFilter (60) : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -95,7 +95,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharC Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#7] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] (5) Scan parquet spark_catalog.default.store_sales @@ -117,30 +117,30 @@ Condition : isnotnull(ss_customer_sk#8) Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] (11) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#12, d_year#13] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] @@ -151,17 +151,17 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) @@ -180,7 +180,7 @@ Condition : (isnotnull(c_customer_sk#19) AND isnotnull(static_invoke(CharVarchar Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Arguments: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25], [c_customer_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#20, 16)) AS c_customer_id#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#21, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#22, 30)) AS c_last_name#25] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [4]: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25] (23) Scan parquet spark_catalog.default.store_sales @@ -191,10 +191,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] Condition : isnotnull(ss_customer_sk#26) @@ -202,30 +202,30 @@ Condition : isnotnull(ss_customer_sk#26) Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#19] Right keys [1]: [ss_customer_sk#26] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, ss_sold_date_sk#28] Input [7]: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25, ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] (29) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#30, d_year#31] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#28] Right keys [1]: [d_date_sk#30] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, d_year#31] Input [7]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, d_year#31] Keys [4]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#27))] @@ -236,10 +236,10 @@ Results [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum# Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] Arguments: hashpartitioning(c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] Keys [4]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31] Functions [1]: [sum(UnscaledValue(ss_net_paid#27))] @@ -250,7 +250,7 @@ Results [4]: [c_customer_id#23 AS customer_id#34, c_first_name#24 AS customer_fi Input [4]: [customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#34] Join type: Inner @@ -271,7 +271,7 @@ Condition : (isnotnull(c_customer_sk#38) AND isnotnull(static_invoke(CharVarchar Input [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] Arguments: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44], [c_customer_sk#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#39, 16)) AS c_customer_id#42, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#40, 20)) AS c_first_name#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#41, 30)) AS c_last_name#44] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [4]: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44] (42) Scan parquet spark_catalog.default.web_sales @@ -282,10 +282,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] Condition : isnotnull(ws_bill_customer_sk#45) @@ -293,30 +293,30 @@ Condition : isnotnull(ws_bill_customer_sk#45) Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#38] Right keys [1]: [ws_bill_customer_sk#45] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, ws_sold_date_sk#47] Input [7]: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44, ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] (48) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#48, d_year#49] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#47] Right keys [1]: [d_date_sk#48] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, d_year#49] Input [7]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, ws_sold_date_sk#47, d_date_sk#48, d_year#49] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, d_year#49] Keys [4]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#46))] @@ -327,17 +327,17 @@ Results [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum# Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] Keys [4]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49] Functions [1]: [sum(UnscaledValue(ws_net_paid#46))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#46))#52] Results [2]: [c_customer_id#42 AS customer_id#53, MakeDecimal(sum(UnscaledValue(ws_net_paid#46))#52,17,2) AS year_total#54] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#53, year_total#54] Condition : (isnotnull(year_total#54) AND (year_total#54 > 0.00)) @@ -345,13 +345,13 @@ Condition : (isnotnull(year_total#54) AND (year_total#54 > 0.00)) Input [2]: [customer_id#53, year_total#54] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#53] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(58) Project [codegen id : 12] Output [7]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54] Input [8]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, customer_id#53, year_total#54] @@ -370,7 +370,7 @@ Condition : (isnotnull(c_customer_sk#55) AND isnotnull(static_invoke(CharVarchar Input [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] Arguments: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61], [c_customer_sk#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#56, 16)) AS c_customer_id#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#57, 20)) AS c_first_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#58, 30)) AS c_last_name#61] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [4]: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61] (63) Scan parquet spark_catalog.default.web_sales @@ -381,10 +381,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] Condition : isnotnull(ws_bill_customer_sk#62) @@ -392,30 +392,30 @@ Condition : isnotnull(ws_bill_customer_sk#62) Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#55] Right keys [1]: [ws_bill_customer_sk#62] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64] Input [7]: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61, ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] (69) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#65, d_year#66] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#64] Right keys [1]: [d_date_sk#65] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#66] Input [7]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#66] Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#63))] @@ -426,10 +426,10 @@ Results [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum# Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66] Functions [1]: [sum(UnscaledValue(ws_net_paid#63))] @@ -440,13 +440,13 @@ Results [2]: [c_customer_id#59 AS customer_id#69, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#69, year_total#70] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#69] Join type: Inner Join condition: (CASE WHEN (year_total#54 > 0.00) THEN (year_total#70 / year_total#54) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#37 / year_total#18) END) -(78) Project [codegen id : 16] +(78) Project [codegen id : 12] Output [3]: [customer_id#34, customer_first_name#35, customer_last_name#36] Input [9]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54, customer_id#69, year_total#70] @@ -458,7 +458,7 @@ Arguments: 100, [customer_first_name#35 ASC NULLS FIRST, customer_id#34 ASC NULL Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometFilter (81) +- CometNativeScan parquet spark_catalog.default.date_dim (80) @@ -474,7 +474,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (83) BroadcastExchange @@ -483,7 +483,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 BroadcastExchange (87) -+- * CometColumnarToRow (86) ++- CometNativeColumnarToRow (86) +- CometFilter (85) +- CometNativeScan parquet spark_catalog.default.date_dim (84) @@ -499,7 +499,7 @@ ReadSchema: struct Input [2]: [d_date_sk#30, d_year#31] Condition : (((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND d_year#31 IN (2001,2002)) AND isnotnull(d_date_sk#30)) -(86) CometColumnarToRow [codegen id : 1] +(86) CometNativeColumnarToRow Input [2]: [d_date_sk#30, d_year#31] (87) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt index 0d59f7b833..5d218998be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -23,23 +23,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -49,24 +49,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -76,19 +76,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -98,7 +98,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/simplified.txt index 9a8a029e1b..95bb7acab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -7,17 +7,17 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] @@ -30,69 +30,65 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -102,25 +98,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt index fefa1d47b1..3c4c40a22f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#28, d_year#29] Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_year#29] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt index 72e39422e0..9ec698fdca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt index fefa1d47b1..3c4c40a22f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#28, d_year#29] Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#28, d_year#29] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 72e39422e0..9ec698fdca 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 @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/explain.txt index 9566cf6e23..cda8563e02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (127) +CometNativeColumnarToRow (127) +- CometTakeOrderedAndProject (126) +- CometProject (125) +- CometSortMergeJoin (124) @@ -25,7 +25,7 @@ : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -136,10 +136,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Condition : isnotnull(cs_item_sk#1) @@ -158,33 +158,33 @@ Condition : ((((((isnotnull(i_category#11) AND (static_invoke(CharVarcharCodegen Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (8) BroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (11) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#13, d_year#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] @@ -236,36 +236,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(24) ColumnarToRow [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) (26) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#22] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] (29) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#32, d_year#33] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] @@ -317,36 +317,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 9] +(42) ColumnarToRow [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -(43) Filter [codegen id : 9] +(43) Filter [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Condition : isnotnull(ws_item_sk#41) (44) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -(45) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_item_sk#41] Right keys [1]: [i_item_sk#46] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(46) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] (47) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#51, d_year#52] -(48) BroadcastHashJoin [codegen id : 9] +(48) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#45] Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(49) Project [codegen id : 9] +(49) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] @@ -443,36 +443,36 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 12] +(70) ColumnarToRow [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -(71) Filter [codegen id : 12] +(71) Filter [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] Condition : isnotnull(cs_item_sk#64) (72) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -(73) BroadcastHashJoin [codegen id : 12] +(73) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_item_sk#64] Right keys [1]: [i_item_sk#70] Join type: Inner Join condition: None -(74) Project [codegen id : 12] +(74) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] (75) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#75, d_year#76] -(76) BroadcastHashJoin [codegen id : 12] +(76) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#68] Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(77) Project [codegen id : 12] +(77) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] @@ -508,36 +508,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 15] +(85) ColumnarToRow [codegen id : 5] Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -(86) Filter [codegen id : 15] +(86) Filter [codegen id : 5] Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] Condition : isnotnull(ss_item_sk#83) (87) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] -(88) BroadcastHashJoin [codegen id : 15] +(88) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#83] Right keys [1]: [i_item_sk#88] Join type: Inner Join condition: None -(89) Project [codegen id : 15] +(89) Project [codegen id : 5] Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] (90) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#93, d_year#94] -(91) BroadcastHashJoin [codegen id : 15] +(91) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#87] Right keys [1]: [d_date_sk#93] Join type: Inner Join condition: None -(92) Project [codegen id : 15] +(92) Project [codegen id : 5] Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_date_sk#93, d_year#94] @@ -573,36 +573,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 18] +(100) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -(101) Filter [codegen id : 18] +(101) Filter [codegen id : 6] Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] Condition : isnotnull(ws_item_sk#101) (102) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -(103) BroadcastHashJoin [codegen id : 18] +(103) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#101] Right keys [1]: [i_item_sk#106] Join type: Inner Join condition: None -(104) Project [codegen id : 18] +(104) Project [codegen id : 6] Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] Input [10]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] (105) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#111, d_year#112] -(106) BroadcastHashJoin [codegen id : 18] +(106) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#105] Right keys [1]: [d_date_sk#111] Join type: Inner Join condition: None -(107) Project [codegen id : 18] +(107) Project [codegen id : 6] Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] Input [11]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] @@ -688,14 +688,14 @@ Arguments: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#127 ASC NULLS FIRST,sales_amt_diff#128 ASC NULLS FIRST], output=[prev_year#123,year#124,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#125,curr_yr_cnt#126,sales_cnt_diff#127,sales_amt_diff#128]), [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128], 100, 0, [sales_cnt_diff#127 ASC NULLS FIRST, sales_amt_diff#128 ASC NULLS FIRST], [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] -(127) CometColumnarToRow [codegen id : 19] +(127) CometNativeColumnarToRow Input [10]: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (131) -+- * CometColumnarToRow (130) ++- CometNativeColumnarToRow (130) +- CometFilter (129) +- CometNativeScan parquet spark_catalog.default.date_dim (128) @@ -711,7 +711,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (131) BroadcastExchange @@ -724,7 +724,7 @@ Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN d Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 BroadcastExchange (135) -+- * CometColumnarToRow (134) ++- CometNativeColumnarToRow (134) +- CometFilter (133) +- CometNativeScan parquet spark_catalog.default.date_dim (132) @@ -740,7 +740,7 @@ ReadSchema: struct Input [2]: [d_date_sk#75, d_year#76] Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) -(134) CometColumnarToRow [codegen id : 1] +(134) CometNativeColumnarToRow Input [2]: [d_date_sk#75, d_year#76] (135) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt index 9f6954be38..42106a0ffd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometSortMergeJoin @@ -25,16 +25,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -55,12 +55,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -81,12 +81,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -117,16 +117,16 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -147,12 +147,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -173,12 +173,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/simplified.txt index aaf98f4b09..84b015b466 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/simplified.txt @@ -1,173 +1,165 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - 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 [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 [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] +CometNativeColumnarToRow + CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + 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 [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 [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] - 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] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometExchange [cr_order_number,cr_item_sk] #7 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (2) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometExchange [sr_ticket_number,sr_item_sk] #9 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (3) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometExchange [wr_order_number,wr_item_sk] #11 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 + 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] #14 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] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (4) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (5) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (6) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt index ea5d8ded80..9ca52fee2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt index d0520c8b63..0e2370eea4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt index ea5d8ded80..9ca52fee2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange 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 d0520c8b63..0e2370eea4 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 @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/explain.txt index 6df18600ea..dcdee90948 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/explain.txt @@ -1,19 +1,19 @@ == Physical Plan == -* CometColumnarToRow (110) +CometNativeColumnarToRow (110) +- CometTakeOrderedAndProject (109) +- CometHashAggregate (108) +- CometColumnarExchange (107) +- * HashAggregate (106) +- Union (105) :- * HashAggregate (90) - : +- * CometColumnarToRow (89) + : +- CometNativeColumnarToRow (89) : +- CometColumnarExchange (88) : +- * HashAggregate (87) : +- Union (86) : :- * Project (32) : : +- * BroadcastHashJoin LeftOuter BuildRight (31) : : :- * HashAggregate (16) - : : : +- * CometColumnarToRow (15) + : : : +- CometNativeColumnarToRow (15) : : : +- CometColumnarExchange (14) : : : +- * HashAggregate (13) : : : +- * Project (12) @@ -25,12 +25,12 @@ : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : +- ReusedExchange (4) : : : +- BroadcastExchange (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometFilter (8) : : : +- CometNativeScan parquet spark_catalog.default.store (7) : : +- BroadcastExchange (30) : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) + : : +- CometNativeColumnarToRow (28) : : +- CometColumnarExchange (27) : : +- * HashAggregate (26) : : +- * Project (25) @@ -46,7 +46,7 @@ : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) : : :- BroadcastExchange (42) : : : +- * HashAggregate (41) - : : : +- * CometColumnarToRow (40) + : : : +- CometNativeColumnarToRow (40) : : : +- CometColumnarExchange (39) : : : +- * HashAggregate (38) : : : +- * Project (37) @@ -55,7 +55,7 @@ : : : : +- Scan parquet spark_catalog.default.catalog_sales (33) : : : +- ReusedExchange (35) : : +- * HashAggregate (51) - : : +- * CometColumnarToRow (50) + : : +- CometNativeColumnarToRow (50) : : +- CometColumnarExchange (49) : : +- * HashAggregate (48) : : +- * Project (47) @@ -66,7 +66,7 @@ : +- * Project (85) : +- * BroadcastHashJoin LeftOuter BuildRight (84) : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) + : : +- CometNativeColumnarToRow (68) : : +- CometColumnarExchange (67) : : +- * HashAggregate (66) : : +- * Project (65) @@ -78,12 +78,12 @@ : : : : +- Scan parquet spark_catalog.default.web_sales (54) : : : +- ReusedExchange (57) : : +- BroadcastExchange (63) - : : +- * CometColumnarToRow (62) + : : +- CometNativeColumnarToRow (62) : : +- CometFilter (61) : : +- CometNativeScan parquet spark_catalog.default.web_page (60) : +- BroadcastExchange (83) : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) + : +- CometNativeColumnarToRow (81) : +- CometColumnarExchange (80) : +- * HashAggregate (79) : +- * Project (78) @@ -96,18 +96,18 @@ : : +- ReusedExchange (73) : +- ReusedExchange (76) :- * HashAggregate (97) - : +- * CometColumnarToRow (96) + : +- CometNativeColumnarToRow (96) : +- CometColumnarExchange (95) : +- * HashAggregate (94) : +- * HashAggregate (93) - : +- * CometColumnarToRow (92) + : +- CometNativeColumnarToRow (92) : +- ReusedExchange (91) +- * HashAggregate (104) - +- * CometColumnarToRow (103) + +- CometNativeColumnarToRow (103) +- CometColumnarExchange (102) +- * HashAggregate (101) +- * HashAggregate (100) - +- * CometColumnarToRow (99) + +- CometNativeColumnarToRow (99) +- ReusedExchange (98) @@ -119,23 +119,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] @@ -150,24 +150,24 @@ ReadSchema: struct Input [1]: [s_store_sk#7] Condition : isnotnull(s_store_sk#7) -(9) CometColumnarToRow [codegen id : 2] +(9) CometNativeColumnarToRow Input [1]: [s_store_sk#7] (10) BroadcastExchange Input [1]: [s_store_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(12) Project [codegen id : 3] +(12) Project [codegen id : 1] Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Keys [1]: [s_store_sk#7] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] @@ -178,10 +178,10 @@ Results [3]: [s_store_sk#7, sum#10, sum#11] Input [3]: [s_store_sk#7, sum#10, sum#11] Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(15) CometColumnarToRow [codegen id : 8] +(15) CometNativeColumnarToRow Input [3]: [s_store_sk#7, sum#10, sum#11] -(16) HashAggregate [codegen id : 8] +(16) HashAggregate [codegen id : 4] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] @@ -196,40 +196,40 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Condition : isnotnull(sr_store_sk#16) (20) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#20] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] (23) ReusedExchange [Reuses operator id: 10] Output [1]: [s_store_sk#21] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_store_sk#16] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Keys [1]: [s_store_sk#21] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] @@ -240,10 +240,10 @@ Results [3]: [s_store_sk#21, sum#24, sum#25] Input [3]: [s_store_sk#21, sum#24, sum#25] Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#21, sum#24, sum#25] -(29) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 3] Input [3]: [s_store_sk#21, sum#24, sum#25] Keys [1]: [s_store_sk#21] Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] @@ -254,13 +254,13 @@ Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26 Input [3]: [s_store_sk#21, returns#28, profit_loss#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#7] Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 4] Output [5]: [store channel AS channel#30, s_store_sk#7 AS id#31, sales#14, coalesce(returns#28, 0.00) AS returns#32, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#33] Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] @@ -271,23 +271,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 5] Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] (35) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#38] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#37] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 5] Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] -(38) HashAggregate [codegen id : 10] +(38) HashAggregate [codegen id : 5] Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Keys [1]: [cs_call_center_sk#34] Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] @@ -298,10 +298,10 @@ Results [3]: [cs_call_center_sk#34, sum#41, sum#42] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometColumnarToRow [codegen id : 11] +(40) CometNativeColumnarToRow Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 6] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Keys [1]: [cs_call_center_sk#34] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] @@ -319,23 +319,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] +(44) ColumnarToRow [codegen id : 7] Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] (45) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#50] -(46) BroadcastHashJoin [codegen id : 13] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cr_returned_date_sk#49] Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(47) Project [codegen id : 13] +(47) Project [codegen id : 7] Output [2]: [cr_return_amount#47, cr_net_loss#48] Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] -(48) HashAggregate [codegen id : 13] +(48) HashAggregate [codegen id : 7] Input [2]: [cr_return_amount#47, cr_net_loss#48] Keys: [] Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] @@ -346,7 +346,7 @@ Results [2]: [sum#53, sum#54] Input [2]: [sum#53, sum#54] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) CometColumnarToRow +(50) CometNativeColumnarToRow Input [2]: [sum#53, sum#54] (51) HashAggregate @@ -356,11 +356,11 @@ Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_ne Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] -(52) BroadcastNestedLoopJoin [codegen id : 14] +(52) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 8] Output [5]: [catalog channel AS channel#59, cs_call_center_sk#34 AS id#60, sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#61] Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] @@ -372,23 +372,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 17] +(55) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -(56) Filter [codegen id : 17] +(56) Filter [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] Condition : isnotnull(ws_web_page_sk#62) (57) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#66] -(58) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#65] Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(59) Project [codegen id : 9] Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] @@ -403,24 +403,24 @@ ReadSchema: struct Input [1]: [wp_web_page_sk#67] Condition : isnotnull(wp_web_page_sk#67) -(62) CometColumnarToRow [codegen id : 16] +(62) CometNativeColumnarToRow Input [1]: [wp_web_page_sk#67] (63) BroadcastExchange Input [1]: [wp_web_page_sk#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(64) BroadcastHashJoin [codegen id : 17] +(64) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_web_page_sk#62] Right keys [1]: [wp_web_page_sk#67] Join type: Inner Join condition: None -(65) Project [codegen id : 17] +(65) Project [codegen id : 9] Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 9] Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Keys [1]: [wp_web_page_sk#67] Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] @@ -431,10 +431,10 @@ Results [3]: [wp_web_page_sk#67, sum#70, sum#71] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometColumnarToRow [codegen id : 22] +(68) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -(69) HashAggregate [codegen id : 22] +(69) HashAggregate [codegen id : 12] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Keys [1]: [wp_web_page_sk#67] Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] @@ -449,40 +449,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -(72) Filter [codegen id : 20] +(72) Filter [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] Condition : isnotnull(wr_web_page_sk#76) (73) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#80] -(74) BroadcastHashJoin [codegen id : 20] +(74) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_returned_date_sk#79] Right keys [1]: [d_date_sk#80] Join type: Inner Join condition: None -(75) Project [codegen id : 20] +(75) Project [codegen id : 10] Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] (76) ReusedExchange [Reuses operator id: 63] Output [1]: [wp_web_page_sk#81] -(77) BroadcastHashJoin [codegen id : 20] +(77) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_web_page_sk#76] Right keys [1]: [wp_web_page_sk#81] Join type: Inner Join condition: None -(78) Project [codegen id : 20] +(78) Project [codegen id : 10] Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -(79) HashAggregate [codegen id : 20] +(79) HashAggregate [codegen id : 10] Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Keys [1]: [wp_web_page_sk#81] Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] @@ -493,10 +493,10 @@ Results [3]: [wp_web_page_sk#81, sum#84, sum#85] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(81) CometColumnarToRow [codegen id : 21] +(81) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -(82) HashAggregate [codegen id : 21] +(82) HashAggregate [codegen id : 11] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Keys [1]: [wp_web_page_sk#81] Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] @@ -507,19 +507,19 @@ Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77) Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 22] +(84) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wp_web_page_sk#67] Right keys [1]: [wp_web_page_sk#81] Join type: LeftOuter Join condition: None -(85) Project [codegen id : 22] +(85) Project [codegen id : 12] Output [5]: [web channel AS channel#90, wp_web_page_sk#67 AS id#91, sales#74, coalesce(returns#88, 0.00) AS returns#92, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#93] Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] (86) Union -(87) HashAggregate [codegen id : 23] +(87) HashAggregate [codegen id : 13] Input [5]: [channel#30, id#31, sales#14, returns#32, profit#33] Keys [2]: [channel#30, id#31] Functions [3]: [partial_sum(sales#14), partial_sum(returns#32), partial_sum(profit#33)] @@ -530,10 +530,10 @@ Results [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(89) CometColumnarToRow [codegen id : 24] +(89) CometNativeColumnarToRow Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(90) HashAggregate [codegen id : 24] +(90) HashAggregate [codegen id : 14] Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Keys [2]: [channel#30, id#31] Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] @@ -543,17 +543,17 @@ Results [5]: [channel#30, id#31, cast(sum(sales#14)#106 as decimal(37,2)) AS sal (91) ReusedExchange [Reuses operator id: 88] Output [8]: [channel#112, id#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -(92) CometColumnarToRow [codegen id : 48] +(92) CometNativeColumnarToRow Input [8]: [channel#112, id#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -(93) HashAggregate [codegen id : 48] +(93) HashAggregate [codegen id : 28] Input [8]: [channel#112, id#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] Keys [2]: [channel#112, id#113] Functions [3]: [sum(sales#120), sum(returns#121), sum(profit#122)] Aggregate Attributes [3]: [sum(sales#120)#106, sum(returns#121)#107, sum(profit#122)#108] Results [4]: [channel#112, sum(sales#120)#106 AS sales#123, sum(returns#121)#107 AS returns#124, sum(profit#122)#108 AS profit#125] -(94) HashAggregate [codegen id : 48] +(94) HashAggregate [codegen id : 28] Input [4]: [channel#112, sales#123, returns#124, profit#125] Keys [1]: [channel#112] Functions [3]: [partial_sum(sales#123), partial_sum(returns#124), partial_sum(profit#125)] @@ -564,10 +564,10 @@ Results [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, Input [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] Arguments: hashpartitioning(channel#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(96) CometColumnarToRow [codegen id : 49] +(96) CometNativeColumnarToRow Input [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -(97) HashAggregate [codegen id : 49] +(97) HashAggregate [codegen id : 29] Input [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] Keys [1]: [channel#112] Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] @@ -577,17 +577,17 @@ Results [5]: [channel#112, null AS id#141, sum(sales#123)#138 AS sales#142, sum( (98) ReusedExchange [Reuses operator id: 88] Output [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -(99) CometColumnarToRow [codegen id : 73] +(99) CometNativeColumnarToRow Input [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -(100) HashAggregate [codegen id : 73] +(100) HashAggregate [codegen id : 43] Input [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] Keys [2]: [channel#145, id#146] Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] Aggregate Attributes [3]: [sum(sales#153)#106, sum(returns#154)#107, sum(profit#155)#108] Results [3]: [sum(sales#153)#106 AS sales#156, sum(returns#154)#107 AS returns#157, sum(profit#155)#108 AS profit#158] -(101) HashAggregate [codegen id : 73] +(101) HashAggregate [codegen id : 43] Input [3]: [sales#156, returns#157, profit#158] Keys: [] Functions [3]: [partial_sum(sales#156), partial_sum(returns#157), partial_sum(profit#158)] @@ -598,10 +598,10 @@ Results [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Input [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(103) CometColumnarToRow [codegen id : 74] +(103) CometNativeColumnarToRow Input [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -(104) HashAggregate [codegen id : 74] +(104) HashAggregate [codegen id : 44] Input [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] Keys: [] Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] @@ -610,7 +610,7 @@ Results [5]: [null AS channel#174, null AS id#175, sum(sales#156)#171 AS sales#1 (105) Union -(106) HashAggregate [codegen id : 75] +(106) HashAggregate [codegen id : 45] Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Functions: [] @@ -630,14 +630,14 @@ Functions: [] Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#30 ASC NULLS FIRST,id#31 ASC NULLS FIRST], output=[channel#30,id#31,sales#109,returns#110,profit#111]), [channel#30, id#31, sales#109, returns#110, profit#111], 100, 0, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30, id#31, sales#109, returns#110, profit#111] -(110) CometColumnarToRow [codegen id : 76] +(110) CometNativeColumnarToRow Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (115) -+- * CometColumnarToRow (114) ++- CometNativeColumnarToRow (114) +- CometProject (113) +- CometFilter (112) +- CometNativeScan parquet spark_catalog.default.date_dim (111) @@ -658,7 +658,7 @@ Condition : (((isnotnull(d_date#179) AND (d_date#179 >= 1998-08-04)) AND (d_date Input [2]: [d_date_sk#6, d_date#179] Arguments: [d_date_sk#6], [d_date_sk#6] -(114) CometColumnarToRow [codegen id : 1] +(114) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (115) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt index 81b485e6b3..178b946ecb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt @@ -1,18 +1,18 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Project : : +- BroadcastHashJoin : : :- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -24,22 +24,22 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -51,19 +51,19 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -72,12 +72,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -86,14 +86,14 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -105,17 +105,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_page : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -127,27 +127,27 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Project : : +- BroadcastHashJoin : : :- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -159,22 +159,22 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -186,19 +186,19 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -207,12 +207,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -221,14 +221,14 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -240,17 +240,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_page : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -262,27 +262,27 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -294,22 +294,22 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -321,19 +321,19 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -342,12 +342,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -356,14 +356,14 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- BroadcastHashJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -375,17 +375,17 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -397,12 +397,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/simplified.txt index 6f40fcdddf..9a9dde3ef9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/simplified.txt @@ -1,180 +1,172 @@ -WholeStageCodegen (76) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (24) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (45) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (14) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (13) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_store_sk] #3 + WholeStageCodegen (1) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_store_sk] #7 + WholeStageCodegen (2) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #9 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow + InputAdapter + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (8) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow + CometNativeColumnarToRow + CometColumnarExchange [cs_call_center_sk] #9 + WholeStageCodegen (5) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (7) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (12) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [wp_web_page_sk] #11 + WholeStageCodegen (9) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_web_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow + InputAdapter + BroadcastExchange #12 + CometNativeColumnarToRow + CometFilter [wp_web_page_sk] + CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (11) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - CometColumnarExchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + CometNativeColumnarToRow + CometColumnarExchange [wp_web_page_sk] #14 + WholeStageCodegen (10) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Filter [wr_web_page_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #15 - WholeStageCodegen (48) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #16 - WholeStageCodegen (73) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [wp_web_page_sk] #12 + WholeStageCodegen (29) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel] #15 + WholeStageCodegen (28) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (44) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #16 + WholeStageCodegen (43) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt index 9554c6732e..fe28663345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt @@ -580,7 +580,7 @@ Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -601,7 +601,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt index b4318d03e2..40e2c31863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt @@ -23,7 +23,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -257,7 +257,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt index ac3d312ee8..d559f4bdcb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt @@ -33,12 +33,10 @@ WholeStageCodegen (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt index 9554c6732e..fe28663345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt @@ -580,7 +580,7 @@ Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -601,7 +601,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt index b4318d03e2..40e2c31863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt @@ -23,7 +23,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -257,7 +257,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ac3d312ee8..d559f4bdcb 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 @@ -33,12 +33,10 @@ WholeStageCodegen (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/explain.txt index dbd053cfac..e227adc462 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (70) +- * Project (69) - +- * CometColumnarToRow (68) + +- CometNativeColumnarToRow (68) +- CometSortMergeJoin (67) :- CometProject (45) : +- CometSortMergeJoin (44) @@ -372,7 +372,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner -(68) CometColumnarToRow [codegen id : 4] +(68) CometNativeColumnarToRow Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] (69) Project [codegen id : 4] @@ -387,7 +387,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -403,7 +403,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (74) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt index 65256f2124..af4b7000b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSortMergeJoin :- CometProject : +- CometSortMergeJoin @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/simplified.txt index 4711669981..e4f8f19efa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (4) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] @@ -24,11 +24,9 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt index 82a5357f89..8e3014f399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt index 273db28e77..ef0945f5d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt @@ -21,11 +21,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.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt index 82a5357f89..8e3014f399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort 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 273db28e77..ef0945f5d6 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 @@ -21,11 +21,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.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/explain.txt index 0807f06c94..45c86665f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (119) +CometNativeColumnarToRow (119) +- CometTakeOrderedAndProject (118) +- CometHashAggregate (117) +- CometExchange (116) @@ -653,14 +653,14 @@ Functions: [] Input [5]: [channel#74, id#75, sales#135, returns#136, profit#137] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#74 ASC NULLS FIRST,id#75 ASC NULLS FIRST], output=[channel#74,id#75,sales#135,returns#136,profit#137]), [channel#74, id#75, sales#135, returns#136, profit#137], 100, 0, [channel#74 ASC NULLS FIRST, id#75 ASC NULLS FIRST], [channel#74, id#75, sales#135, returns#136, profit#137] -(119) CometColumnarToRow [codegen id : 10] +(119) CometNativeColumnarToRow Input [5]: [channel#74, id#75, sales#135, returns#136, profit#137] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometNativeScan parquet spark_catalog.default.date_dim (120) @@ -681,7 +681,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (124) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt index f752943c47..da58e863bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -28,7 +28,7 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,7 +160,7 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -292,7 +292,7 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/simplified.txt index cfce9e20a1..37dfa17b65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/simplified.txt @@ -1,137 +1,133 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometExchange [channel,id,sales,returns,profit] #1 CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - 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] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [cp_catalog_page_id] #11 - 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] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #12 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - 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] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #16 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [s_store_id] #3 + 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] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 + WholeStageCodegen (1) + Filter [ss_store_sk,ss_item_sk,ss_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #8 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [cp_catalog_page_id] #11 + 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] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #12 + WholeStageCodegen (2) + Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [web_site_id] #15 + 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] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #16 + WholeStageCodegen (3) + Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #19 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange #20 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt index 224275b64b..f4c4562c15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt @@ -648,7 +648,7 @@ Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -669,7 +669,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt index 0d6844d93c..943d6e8d1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,7 +156,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -285,7 +285,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt index d27ab827e1..dce81e1279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #6 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt index 224275b64b..f4c4562c15 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt @@ -648,7 +648,7 @@ Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -669,7 +669,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt index 0d6844d93c..943d6e8d1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,7 +156,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -285,7 +285,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d27ab827e1..dce81e1279 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 @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #6 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/explain.txt index 639c731999..5bd1ed43ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (41) +- * Project (40) +- Window (39) - +- * CometColumnarToRow (38) + +- CometNativeColumnarToRow (38) +- CometSort (37) +- CometExchange (36) +- CometHashAggregate (35) @@ -10,7 +10,7 @@ TakeOrderedAndProject (41) +- * HashAggregate (33) +- Union (32) :- * HashAggregate (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometColumnarExchange (15) : +- * HashAggregate (14) : +- * Project (13) @@ -22,23 +22,23 @@ TakeOrderedAndProject (41) : : : +- Scan parquet spark_catalog.default.web_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) + : +- CometNativeColumnarToRow (19) : +- ReusedExchange (18) +- * HashAggregate (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * HashAggregate (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- ReusedExchange (25) @@ -50,23 +50,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ws_item_sk#1, ws_net_paid#2] Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] @@ -85,24 +85,24 @@ Condition : isnotnull(i_item_sk#6) Input [3]: [i_item_sk#6, i_class#7, i_category#8] Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#7, 50)) AS i_class#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#8, 50)) AS i_category#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#6, i_class#9, i_category#10] (11) BroadcastExchange Input [3]: [i_item_sk#6, i_class#9, i_category#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [ws_net_paid#2, i_class#9, i_category#10] Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [ws_net_paid#2, i_class#9, i_category#10] Keys [2]: [i_category#10, i_class#9] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] @@ -113,10 +113,10 @@ Results [3]: [i_category#10, i_class#9, sum#12] Input [3]: [i_category#10, i_class#9, sum#12] Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [3]: [i_category#10, i_class#9, sum#12] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [3]: [i_category#10, i_class#9, sum#12] Keys [2]: [i_category#10, i_class#9] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] @@ -126,17 +126,17 @@ Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as dec (18) ReusedExchange [Reuses operator id: 15] Output [3]: [i_category#20, i_class#21, sum#22] -(19) CometColumnarToRow [codegen id : 8] +(19) CometNativeColumnarToRow Input [3]: [i_category#20, i_class#21, sum#22] -(20) HashAggregate [codegen id : 8] +(20) HashAggregate [codegen id : 4] Input [3]: [i_category#20, i_class#21, sum#22] Keys [2]: [i_category#20, i_class#21] Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#13] Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#13,17,2) AS total_sum#24, i_category#20] -(21) HashAggregate [codegen id : 8] +(21) HashAggregate [codegen id : 4] Input [2]: [total_sum#24, i_category#20] Keys [1]: [i_category#20] Functions [1]: [partial_sum(total_sum#24)] @@ -147,10 +147,10 @@ Results [3]: [i_category#20, sum#27, isEmpty#28] Input [3]: [i_category#20, sum#27, isEmpty#28] Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 9] +(23) CometNativeColumnarToRow Input [3]: [i_category#20, sum#27, isEmpty#28] -(24) HashAggregate [codegen id : 9] +(24) HashAggregate [codegen id : 5] Input [3]: [i_category#20, sum#27, isEmpty#28] Keys [1]: [i_category#20] Functions [1]: [sum(total_sum#24)] @@ -160,17 +160,17 @@ Results [6]: [sum(total_sum#24)#29 AS total_sum#30, i_category#20, null AS i_cla (25) ReusedExchange [Reuses operator id: 15] Output [3]: [i_category#35, i_class#36, sum#37] -(26) CometColumnarToRow [codegen id : 13] +(26) CometNativeColumnarToRow Input [3]: [i_category#35, i_class#36, sum#37] -(27) HashAggregate [codegen id : 13] +(27) HashAggregate [codegen id : 7] Input [3]: [i_category#35, i_class#36, sum#37] Keys [2]: [i_category#35, i_class#36] Functions [1]: [sum(UnscaledValue(ws_net_paid#38))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#38))#13] Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#38))#13,17,2) AS total_sum#39] -(28) HashAggregate [codegen id : 13] +(28) HashAggregate [codegen id : 7] Input [1]: [total_sum#39] Keys: [] Functions [1]: [partial_sum(total_sum#39)] @@ -181,10 +181,10 @@ Results [2]: [sum#42, isEmpty#43] Input [2]: [sum#42, isEmpty#43] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 14] +(30) CometNativeColumnarToRow Input [2]: [sum#42, isEmpty#43] -(31) HashAggregate [codegen id : 14] +(31) HashAggregate [codegen id : 8] Input [2]: [sum#42, isEmpty#43] Keys: [] Functions [1]: [sum(total_sum#39)] @@ -193,7 +193,7 @@ Results [6]: [sum(total_sum#39)#44 AS total_sum#45, null AS i_category#46, null (32) Union -(33) HashAggregate [codegen id : 15] +(33) HashAggregate [codegen id : 9] Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] Functions: [] @@ -217,14 +217,14 @@ Arguments: hashpartitioning(lochierarchy#19, _w0#51, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51], [lochierarchy#19 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] -(38) CometColumnarToRow [codegen id : 16] +(38) CometNativeColumnarToRow Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] (39) Window Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] -(40) Project [codegen id : 17] +(40) Project [codegen id : 10] Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] @@ -236,7 +236,7 @@ Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometNativeScan parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : (((isnotnull(d_month_seq#53) AND (d_month_seq#53 >= 1212)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#53] Arguments: [d_date_sk#5], [d_date_sk#5] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt index a5b37c422c..d56e25c3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -9,7 +9,7 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,26 +21,26 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -52,26 +52,26 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -83,17 +83,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/simplified.txt index 2ccc8c0c39..ad1b79bc92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/simplified.txt @@ -1,71 +1,65 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (17) + WholeStageCodegen (10) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - 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 (15) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + 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 (9) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #6 - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #7 - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (5) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category] #6 + WholeStageCodegen (4) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (8) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #7 + WholeStageCodegen (7) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt index 7f23c0ebbf..c6cf6c32f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt @@ -210,7 +210,7 @@ Arguments: 100, [lochierarchy#32 DESC NULLS LAST, CASE WHEN (lochierarchy#32 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -231,7 +231,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt index 46c47555a8..13ff1e34cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt index 11e3c03d80..67ea1dbf75 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt @@ -23,12 +23,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt index 7f23c0ebbf..c6cf6c32f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt @@ -210,7 +210,7 @@ Arguments: 100, [lochierarchy#32 DESC NULLS LAST, CASE WHEN (lochierarchy#32 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -231,7 +231,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt index 46c47555a8..13ff1e34cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 11e3c03d80..67ea1dbf75 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 @@ -23,12 +23,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/explain.txt index bb10f1112d..3574fed8a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -* CometColumnarToRow (25) +CometNativeColumnarToRow (25) +- CometSort (24) +- CometColumnarExchange (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -19,7 +19,7 @@ : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.store_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -34,10 +34,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) @@ -56,37 +56,37 @@ Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10 Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -97,10 +97,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -115,14 +115,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] @@ -134,14 +134,14 @@ Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 7] +(25) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.date_dim (26) @@ -162,7 +162,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt index 9ab057d467..b8d60879d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt @@ -1,13 +1,13 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -19,17 +19,17 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/simplified.txt index c1ec019e57..efcfa8e8f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/simplified.txt @@ -1,47 +1,39 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] +CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt index 7aac2f05bf..b807c281f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt @@ -142,7 +142,7 @@ Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt index 3dbaf2e346..91e569a85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt index 46191f59cd..7adce04170 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt @@ -23,12 +23,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt index 7aac2f05bf..b807c281f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt @@ -142,7 +142,7 @@ Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt index 3dbaf2e346..91e569a85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 46191f59cd..7adce04170 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 @@ -23,12 +23,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/explain.txt index 1f2e924269..5dd6d5df9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -11,7 +11,7 @@ TakeOrderedAndProject (43) : :- * Project (24) : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) + : : : :- CometNativeColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : +- BroadcastExchange (9) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- Scan parquet spark_catalog.default.catalog_sales (16) : : +- ReusedExchange (18) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer_address (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) @@ -55,7 +55,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] (6) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] @@ -85,7 +85,7 @@ Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi @@ -98,19 +98,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] (13) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#8 AS customer_sk#10] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#7] @@ -121,19 +121,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#6)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] +(17) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] (18) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#13] -(19) BroadcastHashJoin [codegen id : 6] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(20) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#11 AS customer_sk#14] Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] @@ -143,13 +143,13 @@ Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] Input [1]: [customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customer_sk#10] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 9] +(24) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -168,20 +168,20 @@ Condition : (ca_county#16 IN (Walker County,Richland County,Gaines County,Dougla Input [2]: [ca_address_sk#15, ca_county#16] Arguments: [ca_address_sk#15], [ca_address_sk#15] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [1]: [ca_address_sk#15] (29) BroadcastExchange Input [1]: [ca_address_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#15] Join type: Inner Join condition: None -(31) Project [codegen id : 9] +(31) Project [codegen id : 4] Output [1]: [c_current_cdemo_sk#2] Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#15] @@ -200,24 +200,24 @@ Condition : isnotnull(cd_demo_sk#17) Input [9]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Arguments: [cd_demo_sk#17, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25], [cd_demo_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#18, 1, true, false, true) AS cd_gender#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) AS cd_education_status#28, cd_purchase_estimate#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#22, 10, true, false, true) AS cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -(35) CometColumnarToRow [codegen id : 8] +(35) CometNativeColumnarToRow Input [9]: [cd_demo_sk#17, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] (36) BroadcastExchange Input [9]: [cd_demo_sk#17, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#17] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 4] Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#17, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -(39) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 4] Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [1]: [partial_count(1)] @@ -228,10 +228,10 @@ Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_pur Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#31] Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#31] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 5] Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#31] Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] Functions [1]: [count(1)] @@ -246,7 +246,7 @@ Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FI Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -267,7 +267,7 @@ Condition : (((((isnotnull(d_year#39) AND isnotnull(d_moy#40)) AND (d_year#39 = Input [3]: [d_date_sk#7, d_year#39, d_moy#40] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt index 8aebe6be9a..9091536e96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : :- Project : : +- BroadcastHashJoin : : :- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange @@ -20,12 +20,12 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,7 +37,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -47,17 +47,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/simplified.txt index 11bfb6f6c3..5b1aad4ef8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,13 +13,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -27,18 +27,16 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 Union - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -47,7 +45,7 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -58,17 +56,13 @@ TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purcha ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_county] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] InputAdapter BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/explain.txt index ac9e39c190..d6e74dfa6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/explain.txt @@ -232,7 +232,7 @@ Input [14]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35 Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/simplified.txt index 38f41d10a5..b54bf7704a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index ac9e39c190..d6e74dfa6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -232,7 +232,7 @@ Input [14]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35 Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 38f41d10a5..b54bf7704a 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/explain.txt index 207cd830ed..bbbac9673e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/explain.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject (79) : :- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -25,14 +25,14 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -44,14 +44,14 @@ TakeOrderedAndProject (79) : +- BroadcastExchange (56) : +- * Filter (55) : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (47) : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) + : : :- CometNativeColumnarToRow (41) : : : +- CometProject (40) : : : +- CometFilter (39) : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -62,14 +62,14 @@ TakeOrderedAndProject (79) : +- ReusedExchange (48) +- BroadcastExchange (76) +- * HashAggregate (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometColumnarExchange (73) +- * HashAggregate (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) :- * Project (68) : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) + : :- CometNativeColumnarToRow (62) : : +- CometProject (61) : : +- CometFilter (60) : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -95,7 +95,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] (5) Scan parquet spark_catalog.default.store_sales @@ -117,30 +117,30 @@ Condition : isnotnull(ss_customer_sk#15) Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#15] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] (11) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#20, d_year#21] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#18] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] @@ -151,17 +151,17 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#25, year_total#26] Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) @@ -180,7 +180,7 @@ Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] (23) Scan parquet spark_catalog.default.store_sales @@ -191,10 +191,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Condition : isnotnull(ss_customer_sk#35) @@ -202,30 +202,30 @@ Condition : isnotnull(ss_customer_sk#35) Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#27] Right keys [1]: [ss_customer_sk#35] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] (29) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#40, d_year#41] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#38] Right keys [1]: [d_date_sk#40] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] @@ -236,10 +236,10 @@ Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_pre Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] @@ -250,7 +250,7 @@ Results [5]: [c_customer_id#9 AS customer_id#44, c_first_name#10 AS customer_fir Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#44] Join type: Inner @@ -271,7 +271,7 @@ Condition : (isnotnull(c_customer_sk#49) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] Arguments: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62], [c_customer_sk#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#51, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#52, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#53, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#55, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#56, 50, true, false, true) AS c_email_address#62] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [8]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62] (42) Scan parquet spark_catalog.default.web_sales @@ -282,10 +282,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] Condition : isnotnull(ws_bill_customer_sk#63) @@ -293,30 +293,30 @@ Condition : isnotnull(ws_bill_customer_sk#63) Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#49] Right keys [1]: [ws_bill_customer_sk#63] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] Input [12]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] (48) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#67, d_year#68] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#66] Right keys [1]: [d_date_sk#67] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#68] Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66, d_date_sk#67, d_year#68] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#68] Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] @@ -327,17 +327,17 @@ Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cus Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#71] Results [2]: [c_customer_id#57 AS customer_id#72, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#71,18,2) AS year_total#73] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#72, year_total#73] Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) @@ -345,13 +345,13 @@ Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) Input [2]: [customer_id#72, year_total#73] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#72] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(58) Project [codegen id : 12] Output [8]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#73] Input [9]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, customer_id#72, year_total#73] @@ -370,7 +370,7 @@ Condition : (isnotnull(c_customer_sk#74) AND isnotnull(staticinvoke(class org.ap Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] Arguments: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62], [c_customer_sk#74, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#75, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#76, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#77, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#78, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#79, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#80, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#81, 50, true, false, true) AS c_email_address#62] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [8]: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62] (63) Scan parquet spark_catalog.default.web_sales @@ -381,10 +381,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] Condition : isnotnull(ws_bill_customer_sk#82) @@ -392,30 +392,30 @@ Condition : isnotnull(ws_bill_customer_sk#82) Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#74] Right keys [1]: [ws_bill_customer_sk#82] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] Input [12]: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] (69) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#86, d_year#87] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#85] Right keys [1]: [d_date_sk#86] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#87] Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85, d_date_sk#86, d_year#87] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#87] Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87] Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))] @@ -426,10 +426,10 @@ Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cus Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87] Functions [1]: [sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))] @@ -440,13 +440,13 @@ Results [2]: [c_customer_id#57 AS customer_id#90, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#90, year_total#91] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#25] Right keys [1]: [customer_id#90] Join type: Inner Join condition: (CASE WHEN (year_total#73 > 0.00) THEN (year_total#91 / year_total#73) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#48 / year_total#26) ELSE 0E-20 END) -(78) Project [codegen id : 16] +(78) Project [codegen id : 12] Output [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] Input [10]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#73, customer_id#90, year_total#91] @@ -458,7 +458,7 @@ Arguments: 100, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULL Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometFilter (81) +- CometNativeScan parquet spark_catalog.default.date_dim (80) @@ -474,7 +474,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (83) BroadcastExchange @@ -483,7 +483,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 BroadcastExchange (87) -+- * CometColumnarToRow (86) ++- CometNativeColumnarToRow (86) +- CometFilter (85) +- CometNativeScan parquet spark_catalog.default.date_dim (84) @@ -499,7 +499,7 @@ ReadSchema: struct Input [2]: [d_date_sk#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) -(86) CometColumnarToRow [codegen id : 1] +(86) CometNativeColumnarToRow Input [2]: [d_date_sk#40, d_year#41] (87) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt index 0d59f7b833..5d218998be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -23,23 +23,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -49,24 +49,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -76,19 +76,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -98,7 +98,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/simplified.txt index 00a3e659d4..a15087828c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name,customer_email_address] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] @@ -7,17 +7,17 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] @@ -30,69 +30,65 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -102,25 +98,25 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/explain.txt index 49c1c8e0d7..d4e6edd682 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#38, d_year#39] Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#38, d_year#39] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/simplified.txt index 157d1d587e..81d240b8dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/simplified.txt @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index 49c1c8e0d7..d4e6edd682 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -415,7 +415,7 @@ Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, custo Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#20, d_year#21] Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#20, d_year#21] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#38, d_year#39] Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#38, d_year#39] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 157d1d587e..81d240b8dd 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 @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/explain.txt index 2ebf9db593..bfc7f4c2e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.web_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) @@ -54,37 +54,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt index 6c2a775097..15648408fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/simplified.txt index c129b42cdb..2b82e74b78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt index 410aecff1b..aaf59c8747 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt index 58ac81ba0d..11c87950e6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index 410aecff1b..aaf59c8747 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt index 3f41c97ff5..1812b9e872 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 58ac81ba0d..11c87950e6 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/explain.txt index 908ca52087..36a08f12be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/explain.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject (87) +- * BroadcastHashJoin Inner BuildRight (86) :- * Filter (68) : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) + : +- CometNativeColumnarToRow (66) : +- CometColumnarExchange (65) : +- * HashAggregate (64) : +- * Project (63) @@ -17,12 +17,12 @@ TakeOrderedAndProject (87) : : : +- BroadcastExchange (51) : : : +- * Project (50) : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) + : : : :- CometNativeColumnarToRow (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : +- BroadcastExchange (48) : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) + : : : :- CometNativeColumnarToRow (36) : : : : +- CometHashAggregate (35) : : : : +- CometColumnarExchange (34) : : : : +- * HashAggregate (33) @@ -35,7 +35,7 @@ TakeOrderedAndProject (87) : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : +- BroadcastExchange (27) : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) + : : : : : :- CometNativeColumnarToRow (12) : : : : : : +- CometFilter (11) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : +- BroadcastExchange (25) @@ -47,7 +47,7 @@ TakeOrderedAndProject (87) : : : : : : : +- * ColumnarToRow (14) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) + : : : : : : +- CometNativeColumnarToRow (18) : : : : : : +- CometFilter (17) : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : +- ReusedExchange (22) @@ -64,7 +64,7 @@ TakeOrderedAndProject (87) : : : +- ReusedExchange (43) : : +- BroadcastExchange (58) : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) + : : :- CometNativeColumnarToRow (55) : : : +- CometFilter (54) : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : +- ReusedExchange (56) @@ -72,7 +72,7 @@ TakeOrderedAndProject (87) +- BroadcastExchange (85) +- * Filter (84) +- * HashAggregate (83) - +- * CometColumnarToRow (82) + +- CometNativeColumnarToRow (82) +- CometColumnarExchange (81) +- * HashAggregate (80) +- * Project (79) @@ -96,10 +96,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -114,7 +114,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -143,7 +143,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -172,33 +172,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -303,7 +303,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -313,13 +313,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -327,7 +327,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -344,13 +344,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -416,17 +416,17 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 50] +(70) ColumnarToRow [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -(71) Filter [codegen id : 50] +(71) Filter [codegen id : 28] Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] Condition : isnotnull(ss_item_sk#54) (72) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#35] -(73) BroadcastHashJoin [codegen id : 50] +(73) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -435,30 +435,30 @@ Join condition: None (74) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(75) BroadcastHashJoin [codegen id : 50] +(75) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_item_sk#54] Right keys [1]: [i_item_sk#59] Join type: Inner Join condition: None -(76) Project [codegen id : 50] +(76) Project [codegen id : 28] Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] (77) ReusedExchange [Reuses operator id: 126] Output [1]: [d_date_sk#63] -(78) BroadcastHashJoin [codegen id : 50] +(78) BroadcastHashJoin [codegen id : 28] Left keys [1]: [ss_sold_date_sk#57] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(79) Project [codegen id : 50] +(79) Project [codegen id : 28] Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] -(80) HashAggregate [codegen id : 50] +(80) HashAggregate [codegen id : 28] Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] @@ -469,17 +469,17 @@ Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(82) CometColumnarToRow [codegen id : 51] +(82) CometNativeColumnarToRow Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -(83) HashAggregate [codegen id : 51] +(83) HashAggregate [codegen id : 29] Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] -(84) Filter [codegen id : 51] +(84) Filter [codegen id : 29] Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -487,7 +487,7 @@ Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(Reu Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] -(86) BroadcastHashJoin [codegen id : 52] +(86) BroadcastHashJoin [codegen id : 30] Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join type: Inner @@ -501,7 +501,7 @@ Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * HashAggregate (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometColumnarExchange (105) +- * HashAggregate (104) +- Union (103) @@ -529,19 +529,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 2] +(89) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] (90) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#78] -(91) BroadcastHashJoin [codegen id : 2] +(91) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#77] Right keys [1]: [d_date_sk#78] Join type: Inner Join condition: None -(92) Project [codegen id : 2] +(92) Project [codegen id : 1] Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] @@ -552,19 +552,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] ReadSchema: struct -(94) ColumnarToRow [codegen id : 4] +(94) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] (95) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#84] -(96) BroadcastHashJoin [codegen id : 4] +(96) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#83] Right keys [1]: [d_date_sk#84] Join type: Inner Join condition: None -(97) Project [codegen id : 4] +(97) Project [codegen id : 2] Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] @@ -575,25 +575,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] ReadSchema: struct -(99) ColumnarToRow [codegen id : 6] +(99) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] (100) ReusedExchange [Reuses operator id: 121] Output [1]: [d_date_sk#90] -(101) BroadcastHashJoin [codegen id : 6] +(101) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#89] Right keys [1]: [d_date_sk#90] Join type: Inner Join condition: None -(102) Project [codegen id : 6] +(102) Project [codegen id : 3] Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] (103) Union -(104) HashAggregate [codegen id : 7] +(104) HashAggregate [codegen id : 4] Input [2]: [quantity#79, list_price#80] Keys: [] Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] @@ -604,10 +604,10 @@ Results [2]: [sum#95, count#96] Input [2]: [sum#95, count#96] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(106) CometColumnarToRow [codegen id : 8] +(106) CometNativeColumnarToRow Input [2]: [sum#95, count#96] -(107) HashAggregate [codegen id : 8] +(107) HashAggregate [codegen id : 5] Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] @@ -622,7 +622,7 @@ Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#89 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (112) -+- * CometColumnarToRow (111) ++- CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometNativeScan parquet spark_catalog.default.date_dim (108) @@ -643,7 +643,7 @@ Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subq Input [2]: [d_date_sk#40, d_week_seq#99] Arguments: [d_date_sk#40], [d_date_sk#40] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (112) BroadcastExchange @@ -651,7 +651,7 @@ Input [1]: [d_date_sk#40] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] Subquery:6 Hosting operator id = 109 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* CometColumnarToRow (116) +CometNativeColumnarToRow (116) +- CometProject (115) +- CometFilter (114) +- CometNativeScan parquet spark_catalog.default.date_dim (113) @@ -672,12 +672,12 @@ Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Arguments: [d_week_seq#102], [d_week_seq#102] -(116) CometColumnarToRow [codegen id : 1] +(116) CometNativeColumnarToRow Input [1]: [d_week_seq#102] Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometNativeScan parquet spark_catalog.default.date_dim (117) @@ -698,7 +698,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 < Input [2]: [d_date_sk#24, d_year#103] Arguments: [d_date_sk#24], [d_date_sk#24] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (121) BroadcastExchange @@ -713,7 +713,7 @@ Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquer Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 BroadcastExchange (126) -+- * CometColumnarToRow (125) ++- CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometNativeScan parquet spark_catalog.default.date_dim (122) @@ -734,7 +734,7 @@ Condition : ((isnotnull(d_week_seq#106) AND (d_week_seq#106 = Subquery scalar-su Input [2]: [d_date_sk#63, d_week_seq#106] Arguments: [d_date_sk#63], [d_date_sk#63] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_date_sk#63] (126) BroadcastExchange @@ -742,7 +742,7 @@ Input [1]: [d_date_sk#63] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] Subquery:12 Hosting operator id = 123 Hosting Expression = Subquery scalar-subquery#107, [id=#108] -* CometColumnarToRow (130) +CometNativeColumnarToRow (130) +- CometProject (129) +- CometFilter (128) +- CometNativeScan parquet spark_catalog.default.date_dim (127) @@ -763,7 +763,7 @@ Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] Arguments: [d_week_seq#102], [d_week_seq#102] -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [1]: [d_week_seq#102] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt index c4138c254a..7921f2fa61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject :- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -23,7 +23,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -33,12 +33,12 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -51,11 +51,11 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,12 +63,12 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -81,13 +81,13 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -100,16 +100,16 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,28 +123,28 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -157,13 +157,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -176,16 +176,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -199,20 +199,20 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -221,7 +221,7 @@ TakeOrderedAndProject +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -234,11 +234,11 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -246,12 +246,12 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -264,13 +264,13 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -283,16 +283,16 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -306,28 +306,28 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -340,13 +340,13 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -359,16 +359,16 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -382,20 +382,20 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter : +- Subquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/simplified.txt index b0eae963c3..45d526a696 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/simplified.txt @@ -1,18 +1,18 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) + WholeStageCodegen (30) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [sales] Subquery #4 - WholeStageCodegen (8) + WholeStageCodegen (5) HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #12 - WholeStageCodegen (7) + WholeStageCodegen (4) HashAggregate [quantity,list_price] [sum,count,sum,count] InputAdapter Union - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_quantity,ss_list_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -21,7 +21,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [cs_quantity,cs_list_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [ws_quantity,ws_list_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -40,10 +40,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter ReusedExchange [d_date_sk] #6 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) + WholeStageCodegen (14) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -56,37 +56,33 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter BroadcastExchange #3 - WholeStageCodegen (11) + WholeStageCodegen (6) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #4 - WholeStageCodegen (10) + WholeStageCodegen (5) BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometHashAggregate [brand_id,class_id,category_id] CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) + WholeStageCodegen (3) HashAggregate [brand_id,class_id,category_id] Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -98,23 +94,21 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #7 - WholeStageCodegen (4) + WholeStageCodegen (2) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #8 - WholeStageCodegen (3) + WholeStageCodegen (1) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -126,18 +120,16 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedSubquery [d_date_sk] #3 InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 - WholeStageCodegen (9) + WholeStageCodegen (4) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] @@ -153,10 +145,10 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #11 - WholeStageCodegen (23) + WholeStageCodegen (13) BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter @@ -165,14 +157,14 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #13 - WholeStageCodegen (51) + WholeStageCodegen (29) Filter [sales] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) + WholeStageCodegen (28) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] @@ -185,19 +177,15 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter ReusedExchange [ss_item_sk] #3 InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/explain.txt index 33224acffb..148924425f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#98] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#99) AND (d_year#99 >= 1998)) AND (d_year#99 <= 2 Input [2]: [d_date_sk#26, d_year#99] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subq Input [2]: [d_date_sk#64, d_week_seq#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#64] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#98] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt index f56d229b68..c1b791f357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt @@ -44,11 +44,11 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,11 +197,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -224,7 +224,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,7 +288,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/simplified.txt index fb9abae378..71f9d72e7d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/simplified.txt @@ -39,19 +39,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -131,19 +125,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 33224acffb..148924425f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * CometColumnarToRow (106) ++- CometNativeColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) CometColumnarToRow [codegen id : 1] +(106) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) +CometNativeColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(111) CometColumnarToRow [codegen id : 1] +(111) CometNativeColumnarToRow Input [1]: [d_week_seq#98] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * CometColumnarToRow (115) ++- CometNativeColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#99) AND (d_year#99 >= 1998)) AND (d_year#99 <= 2 Input [2]: [d_date_sk#26, d_year#99] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) CometColumnarToRow [codegen id : 1] +(115) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subq Input [2]: [d_date_sk#64, d_week_seq#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#64] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* CometColumnarToRow (125) +CometNativeColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(125) CometColumnarToRow [codegen id : 1] +(125) CometNativeColumnarToRow Input [1]: [d_week_seq#98] Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/extended.txt index f56d229b68..c1b791f357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/extended.txt @@ -44,11 +44,11 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -135,7 +135,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,11 +197,11 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -224,7 +224,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,7 +288,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 fb9abae378..71f9d72e7d 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 @@ -39,19 +39,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #2 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] CometBroadcastExchange [ss_item_sk] #3 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -70,12 +66,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #7 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] @@ -131,19 +125,15 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_week_seq] + Subquery #6 + CometNativeColumnarToRow + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] ReusedExchange [ss_item_sk] #3 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 CometBroadcastExchange [d_date_sk] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/explain.txt index 4c64b2cef3..cbb2f9535e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/explain.txt @@ -1,18 +1,18 @@ == Physical Plan == -* CometColumnarToRow (139) +CometNativeColumnarToRow (139) +- CometTakeOrderedAndProject (138) +- CometHashAggregate (137) +- CometColumnarExchange (136) +- * HashAggregate (135) +- Union (134) :- * HashAggregate (105) - : +- * CometColumnarToRow (104) + : +- CometNativeColumnarToRow (104) : +- CometColumnarExchange (103) : +- * HashAggregate (102) : +- Union (101) : :- * Filter (68) : : +- * HashAggregate (67) - : : +- * CometColumnarToRow (66) + : : +- CometNativeColumnarToRow (66) : : +- CometColumnarExchange (65) : : +- * HashAggregate (64) : : +- * Project (63) @@ -26,12 +26,12 @@ : : : : +- BroadcastExchange (51) : : : : +- * Project (50) : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * CometColumnarToRow (6) + : : : : :- CometNativeColumnarToRow (6) : : : : : +- CometFilter (5) : : : : : +- CometNativeScan parquet spark_catalog.default.item (4) : : : : +- BroadcastExchange (48) : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : :- * CometColumnarToRow (36) + : : : : :- CometNativeColumnarToRow (36) : : : : : +- CometHashAggregate (35) : : : : : +- CometColumnarExchange (34) : : : : : +- * HashAggregate (33) @@ -44,7 +44,7 @@ : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) : : : : : : +- BroadcastExchange (27) : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * CometColumnarToRow (12) + : : : : : : :- CometNativeColumnarToRow (12) : : : : : : : +- CometFilter (11) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) : : : : : : +- BroadcastExchange (25) @@ -56,7 +56,7 @@ : : : : : : : : +- * ColumnarToRow (14) : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometNativeColumnarToRow (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : : +- ReusedExchange (22) @@ -73,14 +73,14 @@ : : : : +- ReusedExchange (43) : : : +- BroadcastExchange (58) : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * CometColumnarToRow (55) + : : : :- CometNativeColumnarToRow (55) : : : : +- CometFilter (54) : : : : +- CometNativeScan parquet spark_catalog.default.item (53) : : : +- ReusedExchange (56) : : +- ReusedExchange (61) : :- * Filter (84) : : +- * HashAggregate (83) - : : +- * CometColumnarToRow (82) + : : +- CometNativeColumnarToRow (82) : : +- CometColumnarExchange (81) : : +- * HashAggregate (80) : : +- * Project (79) @@ -96,7 +96,7 @@ : : +- ReusedExchange (77) : +- * Filter (100) : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) + : +- CometNativeColumnarToRow (98) : +- CometColumnarExchange (97) : +- * HashAggregate (96) : +- * Project (95) @@ -111,32 +111,32 @@ : : +- ReusedExchange (90) : +- ReusedExchange (93) :- * HashAggregate (112) - : +- * CometColumnarToRow (111) + : +- CometNativeColumnarToRow (111) : +- CometColumnarExchange (110) : +- * HashAggregate (109) : +- * HashAggregate (108) - : +- * CometColumnarToRow (107) + : +- CometNativeColumnarToRow (107) : +- ReusedExchange (106) :- * HashAggregate (119) - : +- * CometColumnarToRow (118) + : +- CometNativeColumnarToRow (118) : +- CometColumnarExchange (117) : +- * HashAggregate (116) : +- * HashAggregate (115) - : +- * CometColumnarToRow (114) + : +- CometNativeColumnarToRow (114) : +- ReusedExchange (113) :- * HashAggregate (126) - : +- * CometColumnarToRow (125) + : +- CometNativeColumnarToRow (125) : +- CometColumnarExchange (124) : +- * HashAggregate (123) : +- * HashAggregate (122) - : +- * CometColumnarToRow (121) + : +- CometNativeColumnarToRow (121) : +- ReusedExchange (120) +- * HashAggregate (133) - +- * CometColumnarToRow (132) + +- CometNativeColumnarToRow (132) +- CometColumnarExchange (131) +- * HashAggregate (130) +- * HashAggregate (129) - +- * CometColumnarToRow (128) + +- CometNativeColumnarToRow (128) +- ReusedExchange (127) @@ -148,10 +148,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 25] +(2) ColumnarToRow [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 25] +(3) Filter [codegen id : 14] Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -166,7 +166,7 @@ ReadSchema: struct -(8) ColumnarToRow [codegen id : 6] +(8) ColumnarToRow [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -(9) Filter [codegen id : 6] +(9) Filter [codegen id : 3] Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] Condition : isnotnull(ss_item_sk#10) @@ -195,7 +195,7 @@ ReadSchema: struct -(14) ColumnarToRow [codegen id : 3] +(14) ColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Filter [codegen id : 3] +(15) Filter [codegen id : 1] Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] Condition : isnotnull(cs_item_sk#17) @@ -224,33 +224,33 @@ ReadSchema: struct -(38) ColumnarToRow [codegen id : 9] +(38) ColumnarToRow [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -(39) Filter [codegen id : 9] +(39) Filter [codegen id : 4] Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] Condition : isnotnull(ws_item_sk#28) (40) ReusedExchange [Reuses operator id: 19] Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] -(41) BroadcastHashJoin [codegen id : 9] +(41) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_item_sk#28] Right keys [1]: [i_item_sk#30] Join type: Inner Join condition: None -(42) Project [codegen id : 9] +(42) Project [codegen id : 4] Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] (43) ReusedExchange [Reuses operator id: 174] Output [1]: [d_date_sk#34] -(44) BroadcastHashJoin [codegen id : 9] +(44) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#29] Right keys [1]: [d_date_sk#34] Join type: Inner Join condition: None -(45) Project [codegen id : 9] +(45) Project [codegen id : 4] Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] @@ -355,7 +355,7 @@ Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] -(47) BroadcastHashJoin [codegen id : 10] +(47) BroadcastHashJoin [codegen id : 5] Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] Join type: LeftSemi @@ -365,13 +365,13 @@ Join condition: None Input [3]: [brand_id#25, class_id#26, category_id#27] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 6] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Right keys [3]: [brand_id#25, class_id#26, category_id#27] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 6] Output [1]: [i_item_sk#6 AS ss_item_sk#35] Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] @@ -379,7 +379,7 @@ Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#2 Input [1]: [ss_item_sk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(52) BroadcastHashJoin [codegen id : 25] +(52) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_item_sk#1] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -396,13 +396,13 @@ ReadSchema: struct cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -468,17 +468,17 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 51] +(70) ColumnarToRow [codegen id : 29] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -(71) Filter [codegen id : 51] +(71) Filter [codegen id : 29] Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] Condition : isnotnull(cs_item_sk#54) (72) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#35] -(73) BroadcastHashJoin [codegen id : 51] +(73) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#54] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -487,30 +487,30 @@ Join condition: None (74) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61] -(75) BroadcastHashJoin [codegen id : 51] +(75) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_item_sk#54] Right keys [1]: [i_item_sk#58] Join type: Inner Join condition: None -(76) Project [codegen id : 51] +(76) Project [codegen id : 29] Output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#59, i_class_id#60, i_category_id#61] Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61] (77) ReusedExchange [Reuses operator id: 169] Output [1]: [d_date_sk#62] -(78) BroadcastHashJoin [codegen id : 51] +(78) BroadcastHashJoin [codegen id : 29] Left keys [1]: [cs_sold_date_sk#57] Right keys [1]: [d_date_sk#62] Join type: Inner Join condition: None -(79) Project [codegen id : 51] +(79) Project [codegen id : 29] Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#59, i_class_id#60, i_category_id#61] Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#59, i_class_id#60, i_category_id#61, d_date_sk#62] -(80) HashAggregate [codegen id : 51] +(80) HashAggregate [codegen id : 29] Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#59, i_class_id#60, i_category_id#61] Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61] Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] @@ -521,17 +521,17 @@ Results [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67 Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] Arguments: hashpartitioning(i_brand_id#59, i_class_id#60, i_category_id#61, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(82) CometColumnarToRow [codegen id : 52] +(82) CometNativeColumnarToRow Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] -(83) HashAggregate [codegen id : 52] +(83) HashAggregate [codegen id : 30] Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61] Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70] Results [6]: [catalog AS channel#71, i_brand_id#59, i_class_id#60, i_category_id#61, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#72, count(1)#70 AS number_sales#73] -(84) Filter [codegen id : 52] +(84) Filter [codegen id : 30] Input [6]: [channel#71, i_brand_id#59, i_class_id#60, i_category_id#61, sales#72, number_sales#73] Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) @@ -543,17 +543,17 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(86) ColumnarToRow [codegen id : 77] +(86) ColumnarToRow [codegen id : 44] Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -(87) Filter [codegen id : 77] +(87) Filter [codegen id : 44] Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] Condition : isnotnull(ws_item_sk#74) (88) ReusedExchange [Reuses operator id: 51] Output [1]: [ss_item_sk#35] -(89) BroadcastHashJoin [codegen id : 77] +(89) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#74] Right keys [1]: [ss_item_sk#35] Join type: LeftSemi @@ -562,30 +562,30 @@ Join condition: None (90) ReusedExchange [Reuses operator id: 58] Output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] -(91) BroadcastHashJoin [codegen id : 77] +(91) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_item_sk#74] Right keys [1]: [i_item_sk#78] Join type: Inner Join condition: None -(92) Project [codegen id : 77] +(92) Project [codegen id : 44] Output [6]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#79, i_class_id#80, i_category_id#81] Input [8]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] (93) ReusedExchange [Reuses operator id: 169] Output [1]: [d_date_sk#82] -(94) BroadcastHashJoin [codegen id : 77] +(94) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ws_sold_date_sk#77] Right keys [1]: [d_date_sk#82] Join type: Inner Join condition: None -(95) Project [codegen id : 77] +(95) Project [codegen id : 44] Output [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#79, i_class_id#80, i_category_id#81] Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#79, i_class_id#80, i_category_id#81, d_date_sk#82] -(96) HashAggregate [codegen id : 77] +(96) HashAggregate [codegen id : 44] Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#79, i_class_id#80, i_category_id#81] Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] @@ -596,23 +596,23 @@ Results [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87 Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(98) CometColumnarToRow [codegen id : 78] +(98) CometNativeColumnarToRow Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] -(99) HashAggregate [codegen id : 78] +(99) HashAggregate [codegen id : 45] Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90] Results [6]: [web AS channel#91, i_brand_id#79, i_class_id#80, i_category_id#81, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#92, count(1)#90 AS number_sales#93] -(100) Filter [codegen id : 78] +(100) Filter [codegen id : 45] Input [6]: [channel#91, i_brand_id#79, i_class_id#80, i_category_id#81, sales#92, number_sales#93] Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) (101) Union -(102) HashAggregate [codegen id : 79] +(102) HashAggregate [codegen id : 46] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] @@ -623,10 +623,10 @@ Results [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97 Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(104) CometColumnarToRow [codegen id : 80] +(104) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(105) HashAggregate [codegen id : 80] +(105) HashAggregate [codegen id : 47] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] @@ -636,17 +636,17 @@ Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(sa (106) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(107) CometColumnarToRow [codegen id : 160] +(107) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(108) HashAggregate [codegen id : 160] +(108) HashAggregate [codegen id : 94] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [5]: [channel#49, i_brand_id#37, i_class_id#38, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(109) HashAggregate [codegen id : 160] +(109) HashAggregate [codegen id : 94] Input [5]: [channel#49, i_brand_id#37, i_class_id#38, sum_sales#102, number_sales#103] Keys [3]: [channel#49, i_brand_id#37, i_class_id#38] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] @@ -657,10 +657,10 @@ Results [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, su Input [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(111) CometColumnarToRow [codegen id : 161] +(111) CometNativeColumnarToRow Input [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] -(112) HashAggregate [codegen id : 161] +(112) HashAggregate [codegen id : 95] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] Keys [3]: [channel#49, i_brand_id#37, i_class_id#38] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] @@ -670,17 +670,17 @@ Results [6]: [channel#49, i_brand_id#37, i_class_id#38, null AS i_category_id#11 (113) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(114) CometColumnarToRow [codegen id : 241] +(114) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(115) HashAggregate [codegen id : 241] +(115) HashAggregate [codegen id : 142] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [4]: [channel#49, i_brand_id#37, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(116) HashAggregate [codegen id : 241] +(116) HashAggregate [codegen id : 142] Input [4]: [channel#49, i_brand_id#37, sum_sales#102, number_sales#103] Keys [2]: [channel#49, i_brand_id#37] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] @@ -691,10 +691,10 @@ Results [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] Input [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] Arguments: hashpartitioning(channel#49, i_brand_id#37, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(118) CometColumnarToRow [codegen id : 242] +(118) CometNativeColumnarToRow Input [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] -(119) HashAggregate [codegen id : 242] +(119) HashAggregate [codegen id : 143] Input [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] Keys [2]: [channel#49, i_brand_id#37] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] @@ -704,17 +704,17 @@ Results [6]: [channel#49, i_brand_id#37, null AS i_class_id#123, null AS i_categ (120) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(121) CometColumnarToRow [codegen id : 322] +(121) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(122) HashAggregate [codegen id : 322] +(122) HashAggregate [codegen id : 190] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [3]: [channel#49, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(123) HashAggregate [codegen id : 322] +(123) HashAggregate [codegen id : 190] Input [3]: [channel#49, sum_sales#102, number_sales#103] Keys [1]: [channel#49] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] @@ -725,10 +725,10 @@ Results [4]: [channel#49, sum#130, isEmpty#131, sum#132] Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] -(125) CometColumnarToRow [codegen id : 323] +(125) CometNativeColumnarToRow Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] -(126) HashAggregate [codegen id : 323] +(126) HashAggregate [codegen id : 191] Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] Keys [1]: [channel#49] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] @@ -738,17 +738,17 @@ Results [6]: [channel#49, null AS i_brand_id#135, null AS i_class_id#136, null A (127) ReusedExchange [Reuses operator id: 103] Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(128) CometColumnarToRow [codegen id : 403] +(128) CometNativeColumnarToRow Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -(129) HashAggregate [codegen id : 403] +(129) HashAggregate [codegen id : 238] Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] Functions [2]: [sum(sales#50), sum(number_sales#51)] Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] Results [2]: [sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] -(130) HashAggregate [codegen id : 403] +(130) HashAggregate [codegen id : 238] Input [2]: [sum_sales#102, number_sales#103] Keys: [] Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] @@ -759,10 +759,10 @@ Results [3]: [sum#143, isEmpty#144, sum#145] Input [3]: [sum#143, isEmpty#144, sum#145] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(132) CometColumnarToRow [codegen id : 404] +(132) CometNativeColumnarToRow Input [3]: [sum#143, isEmpty#144, sum#145] -(133) HashAggregate [codegen id : 404] +(133) HashAggregate [codegen id : 239] Input [3]: [sum#143, isEmpty#144, sum#145] Keys: [] Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] @@ -771,7 +771,7 @@ Results [6]: [null AS channel#148, null AS i_brand_id#149, null AS i_class_id#15 (134) Union -(135) HashAggregate [codegen id : 405] +(135) HashAggregate [codegen id : 240] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] Functions: [] @@ -791,14 +791,14 @@ Functions: [] Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#37 ASC NULLS FIRST,i_class_id#38 ASC NULLS FIRST,i_category_id#39 ASC NULLS FIRST], output=[channel#49,i_brand_id#37,i_class_id#38,i_category_id#39,sum_sales#102,number_sales#103]), [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -(139) CometColumnarToRow [codegen id : 406] +(139) CometNativeColumnarToRow Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] ===== Subqueries ===== Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * HashAggregate (159) -+- * CometColumnarToRow (158) ++- CometNativeColumnarToRow (158) +- CometColumnarExchange (157) +- * HashAggregate (156) +- Union (155) @@ -826,19 +826,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#156), dynamicpruningexpression(ss_sold_date_sk#156 IN dynamicpruning#12)] ReadSchema: struct -(141) ColumnarToRow [codegen id : 2] +(141) ColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156] (142) ReusedExchange [Reuses operator id: 174] Output [1]: [d_date_sk#157] -(143) BroadcastHashJoin [codegen id : 2] +(143) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#156] Right keys [1]: [d_date_sk#157] Join type: Inner Join condition: None -(144) Project [codegen id : 2] +(144) Project [codegen id : 1] Output [2]: [ss_quantity#154 AS quantity#158, ss_list_price#155 AS list_price#159] Input [4]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156, d_date_sk#157] @@ -849,19 +849,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#162), dynamicpruningexpression(cs_sold_date_sk#162 IN dynamicpruning#163)] ReadSchema: struct -(146) ColumnarToRow [codegen id : 4] +(146) ColumnarToRow [codegen id : 2] Input [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162] (147) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#164] -(148) BroadcastHashJoin [codegen id : 4] +(148) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#162] Right keys [1]: [d_date_sk#164] Join type: Inner Join condition: None -(149) Project [codegen id : 4] +(149) Project [codegen id : 2] Output [2]: [cs_quantity#160 AS quantity#165, cs_list_price#161 AS list_price#166] Input [4]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162, d_date_sk#164] @@ -872,25 +872,25 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#169), dynamicpruningexpression(ws_sold_date_sk#169 IN dynamicpruning#163)] ReadSchema: struct -(151) ColumnarToRow [codegen id : 6] +(151) ColumnarToRow [codegen id : 3] Input [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169] (152) ReusedExchange [Reuses operator id: 164] Output [1]: [d_date_sk#170] -(153) BroadcastHashJoin [codegen id : 6] +(153) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#169] Right keys [1]: [d_date_sk#170] Join type: Inner Join condition: None -(154) Project [codegen id : 6] +(154) Project [codegen id : 3] Output [2]: [ws_quantity#167 AS quantity#171, ws_list_price#168 AS list_price#172] Input [4]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169, d_date_sk#170] (155) Union -(156) HashAggregate [codegen id : 7] +(156) HashAggregate [codegen id : 4] Input [2]: [quantity#158, list_price#159] Keys: [] Functions [1]: [partial_avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] @@ -901,10 +901,10 @@ Results [2]: [sum#175, count#176] Input [2]: [sum#175, count#176] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] -(158) CometColumnarToRow [codegen id : 8] +(158) CometNativeColumnarToRow Input [2]: [sum#175, count#176] -(159) HashAggregate [codegen id : 8] +(159) HashAggregate [codegen id : 5] Input [2]: [sum#175, count#176] Keys: [] Functions [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] @@ -915,7 +915,7 @@ Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#156 IN Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163 BroadcastExchange (164) -+- * CometColumnarToRow (163) ++- CometNativeColumnarToRow (163) +- CometProject (162) +- CometFilter (161) +- CometNativeScan parquet spark_catalog.default.date_dim (160) @@ -936,7 +936,7 @@ Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 < Input [2]: [d_date_sk#164, d_year#179] Arguments: [d_date_sk#164], [d_date_sk#164] -(163) CometColumnarToRow [codegen id : 1] +(163) CometNativeColumnarToRow Input [1]: [d_date_sk#164] (164) BroadcastExchange @@ -947,7 +947,7 @@ Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#169 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (169) -+- * CometColumnarToRow (168) ++- CometNativeColumnarToRow (168) +- CometProject (167) +- CometFilter (166) +- CometNativeScan parquet spark_catalog.default.date_dim (165) @@ -968,7 +968,7 @@ Condition : ((((isnotnull(d_year#180) AND isnotnull(d_moy#181)) AND (d_year#180 Input [3]: [d_date_sk#40, d_year#180, d_moy#181] Arguments: [d_date_sk#40], [d_date_sk#40] -(168) CometColumnarToRow [codegen id : 1] +(168) CometNativeColumnarToRow Input [1]: [d_date_sk#40] (169) BroadcastExchange @@ -977,7 +977,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (174) -+- * CometColumnarToRow (173) ++- CometNativeColumnarToRow (173) +- CometProject (172) +- CometFilter (171) +- CometNativeScan parquet spark_catalog.default.date_dim (170) @@ -998,7 +998,7 @@ Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 < Input [2]: [d_date_sk#24, d_year#182] Arguments: [d_date_sk#24], [d_date_sk#24] -(173) CometColumnarToRow [codegen id : 1] +(173) CometNativeColumnarToRow Input [1]: [d_date_sk#24] (174) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt index c6478057d9..0bc07e401e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt @@ -1,18 +1,18 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -32,12 +32,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -47,12 +47,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -65,19 +65,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -90,13 +90,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -109,16 +109,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -132,28 +132,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -166,13 +166,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -185,16 +185,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -208,23 +208,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -239,12 +239,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -257,13 +257,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -276,16 +276,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -299,28 +299,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -333,13 +333,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -352,16 +352,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -375,23 +375,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -406,12 +406,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -424,13 +424,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -443,16 +443,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -466,28 +466,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -500,13 +500,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -519,16 +519,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -542,32 +542,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -577,7 +577,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -587,12 +587,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -602,12 +602,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -620,19 +620,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -645,13 +645,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -664,16 +664,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -687,28 +687,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -721,13 +721,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -740,16 +740,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -763,23 +763,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -794,12 +794,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -812,13 +812,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -831,16 +831,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -854,28 +854,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -888,13 +888,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -907,16 +907,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -930,23 +930,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -961,12 +961,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -979,13 +979,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -998,16 +998,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1021,28 +1021,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -1055,13 +1055,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -1074,16 +1074,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1097,32 +1097,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -1132,7 +1132,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1142,12 +1142,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1157,12 +1157,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1175,19 +1175,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1200,13 +1200,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1219,16 +1219,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1242,28 +1242,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1276,13 +1276,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1295,16 +1295,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1318,23 +1318,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1349,12 +1349,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1367,13 +1367,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1386,16 +1386,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1409,28 +1409,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1443,13 +1443,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1462,16 +1462,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1485,23 +1485,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -1516,12 +1516,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1534,13 +1534,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1553,16 +1553,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1576,28 +1576,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -1610,13 +1610,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -1629,16 +1629,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1652,32 +1652,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Filter : : : +- Subquery : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Union @@ -1687,7 +1687,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1697,12 +1697,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1712,12 +1712,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1730,19 +1730,19 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1755,13 +1755,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1774,16 +1774,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1797,28 +1797,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1831,13 +1831,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -1850,16 +1850,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1873,23 +1873,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : :- Filter : : : +- ReusedSubquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -1904,12 +1904,12 @@ CometColumnarToRow : : : : +- BroadcastExchange : : : : +- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometHashAggregate : : : : : +- CometColumnarExchange : : : : : +- HashAggregate @@ -1922,13 +1922,13 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow + : : : : : : :- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange @@ -1941,16 +1941,16 @@ CometColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1964,28 +1964,28 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -1998,13 +1998,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2017,16 +2017,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2040,23 +2040,23 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -2071,12 +2071,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -2089,13 +2089,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2108,16 +2108,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2131,28 +2131,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2165,13 +2165,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2184,16 +2184,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2207,32 +2207,32 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- Filter : : +- Subquery : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Union @@ -2242,7 +2242,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2252,12 +2252,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2267,12 +2267,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -2285,19 +2285,19 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -2310,13 +2310,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2329,16 +2329,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2352,28 +2352,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2386,13 +2386,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2405,16 +2405,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2428,23 +2428,23 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- Filter : : +- ReusedSubquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -2459,12 +2459,12 @@ CometColumnarToRow : : : +- BroadcastExchange : : : +- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometHashAggregate : : : : +- CometColumnarExchange : : : : +- HashAggregate @@ -2477,13 +2477,13 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow + : : : : : :- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange @@ -2496,16 +2496,16 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2519,28 +2519,28 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2553,13 +2553,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2572,16 +2572,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2595,23 +2595,23 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Filter : +- ReusedSubquery +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -2626,12 +2626,12 @@ CometColumnarToRow : : +- BroadcastExchange : : +- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometColumnarExchange : : : +- HashAggregate @@ -2644,13 +2644,13 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange @@ -2663,16 +2663,16 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2686,28 +2686,28 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometColumnarExchange : : +- HashAggregate @@ -2720,13 +2720,13 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange @@ -2739,16 +2739,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2762,16 +2762,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/simplified.txt index 4458723393..69a4e70593 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/simplified.txt @@ -1,276 +1,266 @@ -WholeStageCodegen (406) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + WholeStageCodegen (240) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Union + WholeStageCodegen (47) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (46) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + InputAdapter + Union + WholeStageCodegen (15) + Filter [sales] + Subquery #3 + WholeStageCodegen (5) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #14 + WholeStageCodegen (4) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (1) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (2) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #15 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (3) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #15 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (14) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + BroadcastExchange #6 + WholeStageCodegen (5) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + CometNativeColumnarToRow + CometHashAggregate [brand_id,class_id,category_id] + CometColumnarExchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (3) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (2) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + InputAdapter + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + BroadcastExchange #11 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (4) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #13 + WholeStageCodegen (13) + BroadcastHashJoin [i_item_sk,ss_item_sk] + InputAdapter + CometNativeColumnarToRow + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (30) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (29) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (45) + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (44) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id] #19 - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #20 - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #21 - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (95) + HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,i_brand_id,i_class_id] #18 + WholeStageCodegen (94) + HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (143) + HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,i_brand_id] #19 + WholeStageCodegen (142) + HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (191) + HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel] #20 + WholeStageCodegen (190) + HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + WholeStageCodegen (239) + HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #21 + WholeStageCodegen (238) + HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] + InputAdapter + CometNativeColumnarToRow + 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/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/explain.txt index 4ac6c99894..798e834945 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/explain.txt @@ -809,7 +809,7 @@ Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#123 IN Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#131 BroadcastExchange (147) -+- * CometColumnarToRow (146) ++- CometNativeColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 < Input [2]: [d_date_sk#132, d_year#133] Arguments: [d_date_sk#132], [d_date_sk#132] -(146) CometColumnarToRow [codegen id : 1] +(146) CometNativeColumnarToRow Input [1]: [d_date_sk#132] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#138 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * CometColumnarToRow (151) ++- CometNativeColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) CometColumnarToRow [codegen id : 1] +(151) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * CometColumnarToRow (156) ++- CometNativeColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#146) AND (d_year#146 >= 1999)) AND (d_year#146 < Input [2]: [d_date_sk#26, d_year#146] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) CometColumnarToRow [codegen id : 1] +(156) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (157) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt index 83deb45432..3bacf3c8bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -219,7 +219,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -283,7 +283,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -360,7 +360,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -424,7 +424,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -497,7 +497,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -525,7 +525,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -547,7 +547,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -611,7 +611,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -688,7 +688,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -752,7 +752,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -829,7 +829,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -893,7 +893,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -966,7 +966,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -994,7 +994,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1016,7 +1016,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1080,7 +1080,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1157,7 +1157,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1221,7 +1221,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1298,7 +1298,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1362,7 +1362,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1435,7 +1435,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1463,7 +1463,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1485,7 +1485,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1549,7 +1549,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1626,7 +1626,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1690,7 +1690,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1767,7 +1767,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1831,7 +1831,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1904,7 +1904,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1932,7 +1932,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1954,7 +1954,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2018,7 +2018,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2095,7 +2095,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2159,7 +2159,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2236,7 +2236,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2300,7 +2300,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/simplified.txt index 758e917335..26ab7f52d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/simplified.txt @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #18 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -56,12 +54,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #5 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -80,12 +76,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 4ac6c99894..798e834945 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -809,7 +809,7 @@ Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#123 IN Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#131 BroadcastExchange (147) -+- * CometColumnarToRow (146) ++- CometNativeColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 < Input [2]: [d_date_sk#132, d_year#133] Arguments: [d_date_sk#132], [d_date_sk#132] -(146) CometColumnarToRow [codegen id : 1] +(146) CometNativeColumnarToRow Input [1]: [d_date_sk#132] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#138 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * CometColumnarToRow (151) ++- CometNativeColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) CometColumnarToRow [codegen id : 1] +(151) CometNativeColumnarToRow Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * CometColumnarToRow (156) ++- CometNativeColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#146) AND (d_year#146 >= 1999)) AND (d_year#146 < Input [2]: [d_date_sk#26, d_year#146] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) CometColumnarToRow [codegen id : 1] +(156) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (157) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/extended.txt index 83deb45432..3bacf3c8bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/extended.txt @@ -28,7 +28,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -219,7 +219,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -283,7 +283,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -360,7 +360,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -424,7 +424,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -497,7 +497,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -525,7 +525,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -547,7 +547,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -611,7 +611,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -688,7 +688,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -752,7 +752,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -829,7 +829,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -893,7 +893,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -966,7 +966,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -994,7 +994,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1016,7 +1016,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1080,7 +1080,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1157,7 +1157,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1221,7 +1221,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1298,7 +1298,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1362,7 +1362,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1435,7 +1435,7 @@ CometColumnarToRow : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1463,7 +1463,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1485,7 +1485,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1549,7 +1549,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1626,7 +1626,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1690,7 +1690,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1767,7 +1767,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1831,7 +1831,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1904,7 +1904,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1932,7 +1932,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1954,7 +1954,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2018,7 +2018,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2095,7 +2095,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2159,7 +2159,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2236,7 +2236,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2300,7 +2300,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 758e917335..26ab7f52d4 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 @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #18 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -56,12 +54,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [ss_item_sk] #5 CometProject [i_item_sk] [ss_item_sk] CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] @@ -80,12 +76,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/explain.txt index 6f109523cf..5469fcd1a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (161) +- Union (160) :- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -21,32 +21,32 @@ TakeOrderedAndProject (161) : : : : : : : +- * ColumnarToRow (2) : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) + : : : : : : +- CometNativeColumnarToRow (7) : : : : : : +- CometProject (6) : : : : : : +- CometFilter (5) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) + : : : : : +- CometNativeColumnarToRow (14) : : : : : +- CometProject (13) : : : : : +- CometFilter (12) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) + : : : : +- CometNativeColumnarToRow (20) : : : : +- CometFilter (19) : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) + : : : +- CometNativeColumnarToRow (27) : : : +- CometProject (26) : : : +- CometFilter (25) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) : : +- ReusedExchange (31) : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometProject (36) : +- CometFilter (35) : +- CometNativeScan parquet spark_catalog.default.item (34) :- * HashAggregate (73) - : +- * CometColumnarToRow (72) + : +- CometNativeColumnarToRow (72) : +- CometColumnarExchange (71) : +- * HashAggregate (70) : +- * Project (69) @@ -68,14 +68,14 @@ TakeOrderedAndProject (161) : : : : : +- ReusedExchange (51) : : : : +- ReusedExchange (54) : : : +- BroadcastExchange (61) - : : : +- * CometColumnarToRow (60) + : : : +- CometNativeColumnarToRow (60) : : : +- CometProject (59) : : : +- CometFilter (58) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (57) : : +- ReusedExchange (64) : +- ReusedExchange (67) :- * HashAggregate (102) - : +- * CometColumnarToRow (101) + : +- CometNativeColumnarToRow (101) : +- CometColumnarExchange (100) : +- * HashAggregate (99) : +- * Project (98) @@ -97,14 +97,14 @@ TakeOrderedAndProject (161) : : : : : +- ReusedExchange (80) : : : : +- ReusedExchange (83) : : : +- BroadcastExchange (90) - : : : +- * CometColumnarToRow (89) + : : : +- CometNativeColumnarToRow (89) : : : +- CometProject (88) : : : +- CometFilter (87) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (86) : : +- ReusedExchange (93) : +- ReusedExchange (96) :- * HashAggregate (131) - : +- * CometColumnarToRow (130) + : +- CometNativeColumnarToRow (130) : +- CometColumnarExchange (129) : +- * HashAggregate (128) : +- * Project (127) @@ -126,14 +126,14 @@ TakeOrderedAndProject (161) : : : : : +- ReusedExchange (109) : : : : +- ReusedExchange (112) : : : +- BroadcastExchange (119) - : : : +- * CometColumnarToRow (118) + : : : +- CometNativeColumnarToRow (118) : : : +- CometProject (117) : : : +- CometFilter (116) : : : +- CometNativeScan parquet spark_catalog.default.customer_address (115) : : +- ReusedExchange (122) : +- ReusedExchange (125) +- * HashAggregate (159) - +- * CometColumnarToRow (158) + +- CometNativeColumnarToRow (158) +- CometColumnarExchange (157) +- * HashAggregate (156) +- * Project (155) @@ -157,7 +157,7 @@ TakeOrderedAndProject (161) : : +- ReusedExchange (144) : +- ReusedExchange (147) +- BroadcastExchange (153) - +- * CometColumnarToRow (152) + +- CometNativeColumnarToRow (152) +- CometFilter (151) +- CometNativeScan parquet spark_catalog.default.item (150) @@ -170,10 +170,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] +(2) ColumnarToRow [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -(3) Filter [codegen id : 7] +(3) Filter [codegen id : 1] Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) @@ -192,20 +192,20 @@ Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarchar Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [2]: [cd_demo_sk#11, cd_dep_count#14] (8) BroadcastExchange Input [2]: [cd_demo_sk#11, cd_dep_count#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 7] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] Join type: Inner Join condition: None -(10) Project [codegen id : 7] +(10) Project [codegen id : 1] Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] @@ -224,20 +224,20 @@ Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#1 Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -(14) CometColumnarToRow [codegen id : 2] +(14) CometNativeColumnarToRow Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] (15) BroadcastExchange Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 7] +(16) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#15] Join type: Inner Join condition: None -(17) Project [codegen id : 7] +(17) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] @@ -252,20 +252,20 @@ ReadSchema: struct Input [1]: [cd_demo_sk#20] Condition : isnotnull(cd_demo_sk#20) -(20) CometColumnarToRow [codegen id : 3] +(20) CometNativeColumnarToRow Input [1]: [cd_demo_sk#20] (21) BroadcastExchange Input [1]: [cd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 7] +(22) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_cdemo_sk#16] Right keys [1]: [cd_demo_sk#20] Join type: Inner Join condition: None -(23) Project [codegen id : 7] +(23) Project [codegen id : 1] Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] @@ -284,33 +284,33 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] -(27) CometColumnarToRow [codegen id : 4] +(27) CometNativeColumnarToRow Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (28) BroadcastExchange Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(29) BroadcastHashJoin [codegen id : 7] +(29) BroadcastHashJoin [codegen id : 1] Left keys [1]: [c_current_addr_sk#17] Right keys [1]: [ca_address_sk#21] Join type: Inner Join condition: None -(30) Project [codegen id : 7] +(30) Project [codegen id : 1] Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] (31) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#26] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#26] Join type: Inner Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 1] Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] @@ -329,24 +329,24 @@ Condition : isnotnull(i_item_sk#27) Input [2]: [i_item_sk#27, i_item_id#28] Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] -(37) CometColumnarToRow [codegen id : 6] +(37) CometNativeColumnarToRow Input [2]: [i_item_sk#27, i_item_id#29] (38) BroadcastExchange Input [2]: [i_item_sk#27, i_item_id#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 7] +(39) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#3] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(40) Project [codegen id : 7] +(40) Project [codegen id : 1] Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] -(41) HashAggregate [codegen id : 7] +(41) HashAggregate [codegen id : 1] Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] @@ -357,10 +357,10 @@ Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, c Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 8] +(43) CometNativeColumnarToRow Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -(44) HashAggregate [codegen id : 8] +(44) HashAggregate [codegen id : 2] Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] @@ -375,49 +375,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(46) ColumnarToRow [codegen id : 15] +(46) ColumnarToRow [codegen id : 3] Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -(47) Filter [codegen id : 15] +(47) Filter [codegen id : 3] Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) (48) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#92, cd_dep_count#93] -(49) BroadcastHashJoin [codegen id : 15] +(49) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_cdemo_sk#84] Right keys [1]: [cd_demo_sk#92] Join type: Inner Join condition: None -(50) Project [codegen id : 15] +(50) Project [codegen id : 3] Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93] Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#92, cd_dep_count#93] (51) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] -(52) BroadcastHashJoin [codegen id : 15] +(52) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_bill_customer_sk#83] Right keys [1]: [c_customer_sk#94] Join type: Inner Join condition: None -(53) Project [codegen id : 15] +(53) Project [codegen id : 3] Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] (54) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#98] -(55) BroadcastHashJoin [codegen id : 15] +(55) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#95] Right keys [1]: [cd_demo_sk#98] Join type: Inner Join condition: None -(56) Project [codegen id : 15] +(56) Project [codegen id : 3] Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97] Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97, cd_demo_sk#98] @@ -436,50 +436,50 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] Arguments: [ca_address_sk#99, ca_state#25, ca_country#101], [ca_address_sk#99, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#100, 2, true, false, true) AS ca_state#25, ca_country#101] -(60) CometColumnarToRow [codegen id : 12] +(60) CometNativeColumnarToRow Input [3]: [ca_address_sk#99, ca_state#25, ca_country#101] (61) BroadcastExchange Input [3]: [ca_address_sk#99, ca_state#25, ca_country#101] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -(62) BroadcastHashJoin [codegen id : 15] +(62) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#96] Right keys [1]: [ca_address_sk#99] Join type: Inner Join condition: None -(63) Project [codegen id : 15] +(63) Project [codegen id : 3] Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101] Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97, ca_address_sk#99, ca_state#25, ca_country#101] (64) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#102] -(65) BroadcastHashJoin [codegen id : 15] +(65) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#91] Right keys [1]: [d_date_sk#102] Join type: Inner Join condition: None -(66) Project [codegen id : 15] +(66) Project [codegen id : 3] Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101] Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101, d_date_sk#102] (67) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#103, i_item_id#29] -(68) BroadcastHashJoin [codegen id : 15] +(68) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_item_sk#85] Right keys [1]: [i_item_sk#103] Join type: Inner Join condition: None -(69) Project [codegen id : 15] +(69) Project [codegen id : 3] Output [10]: [i_item_id#29, ca_country#101, ca_state#25, cast(cs_quantity#86 as decimal(12,2)) AS agg1#34, cast(cs_list_price#87 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#38, cast(c_birth_year#97 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#40] Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101, i_item_sk#103, i_item_id#29] -(70) HashAggregate [codegen id : 15] +(70) HashAggregate [codegen id : 3] Input [10]: [i_item_id#29, ca_country#101, ca_state#25, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] Keys [3]: [i_item_id#29, ca_country#101, ca_state#25] Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] @@ -490,10 +490,10 @@ Results [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#118, count#119, su Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131] Arguments: hashpartitioning(i_item_id#29, ca_country#101, ca_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(72) CometColumnarToRow [codegen id : 16] +(72) CometNativeColumnarToRow Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131] -(73) HashAggregate [codegen id : 16] +(73) HashAggregate [codegen id : 4] Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131] Keys [3]: [i_item_id#29, ca_country#101, ca_state#25] Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] @@ -508,49 +508,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#155), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(75) ColumnarToRow [codegen id : 23] +(75) ColumnarToRow [codegen id : 5] Input [9]: [cs_bill_customer_sk#147, cs_bill_cdemo_sk#148, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155] -(76) Filter [codegen id : 23] +(76) Filter [codegen id : 5] Input [9]: [cs_bill_customer_sk#147, cs_bill_cdemo_sk#148, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155] Condition : ((isnotnull(cs_bill_cdemo_sk#148) AND isnotnull(cs_bill_customer_sk#147)) AND isnotnull(cs_item_sk#149)) (77) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#156, cd_dep_count#157] -(78) BroadcastHashJoin [codegen id : 23] +(78) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_bill_cdemo_sk#148] Right keys [1]: [cd_demo_sk#156] Join type: Inner Join condition: None -(79) Project [codegen id : 23] +(79) Project [codegen id : 5] Output [9]: [cs_bill_customer_sk#147, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157] Input [11]: [cs_bill_customer_sk#147, cs_bill_cdemo_sk#148, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_demo_sk#156, cd_dep_count#157] (80) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_addr_sk#160, c_birth_year#161] -(81) BroadcastHashJoin [codegen id : 23] +(81) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_bill_customer_sk#147] Right keys [1]: [c_customer_sk#158] Join type: Inner Join condition: None -(82) Project [codegen id : 23] +(82) Project [codegen id : 5] Output [11]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_current_cdemo_sk#159, c_current_addr_sk#160, c_birth_year#161] Input [13]: [cs_bill_customer_sk#147, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_addr_sk#160, c_birth_year#161] (83) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#162] -(84) BroadcastHashJoin [codegen id : 23] +(84) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_cdemo_sk#159] Right keys [1]: [cd_demo_sk#162] Join type: Inner Join condition: None -(85) Project [codegen id : 23] +(85) Project [codegen id : 5] Output [10]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_current_addr_sk#160, c_birth_year#161] Input [12]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_current_cdemo_sk#159, c_current_addr_sk#160, c_birth_year#161, cd_demo_sk#162] @@ -569,50 +569,50 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [3]: [ca_address_sk#163, ca_state#164, ca_country#165] Arguments: [ca_address_sk#163, ca_country#165], [ca_address_sk#163, ca_country#165] -(89) CometColumnarToRow [codegen id : 20] +(89) CometNativeColumnarToRow Input [2]: [ca_address_sk#163, ca_country#165] (90) BroadcastExchange Input [2]: [ca_address_sk#163, ca_country#165] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -(91) BroadcastHashJoin [codegen id : 23] +(91) BroadcastHashJoin [codegen id : 5] Left keys [1]: [c_current_addr_sk#160] Right keys [1]: [ca_address_sk#163] Join type: Inner Join condition: None -(92) Project [codegen id : 23] +(92) Project [codegen id : 5] Output [10]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_birth_year#161, ca_country#165] Input [12]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_current_addr_sk#160, c_birth_year#161, ca_address_sk#163, ca_country#165] (93) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#166] -(94) BroadcastHashJoin [codegen id : 23] +(94) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#155] Right keys [1]: [d_date_sk#166] Join type: Inner Join condition: None -(95) Project [codegen id : 23] +(95) Project [codegen id : 5] Output [9]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cd_dep_count#157, c_birth_year#161, ca_country#165] Input [11]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_birth_year#161, ca_country#165, d_date_sk#166] (96) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#167, i_item_id#29] -(97) BroadcastHashJoin [codegen id : 23] +(97) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_item_sk#149] Right keys [1]: [i_item_sk#167] Join type: Inner Join condition: None -(98) Project [codegen id : 23] +(98) Project [codegen id : 5] Output [9]: [i_item_id#29, ca_country#165, cast(cs_quantity#150 as decimal(12,2)) AS agg1#34, cast(cs_list_price#151 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#153 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#152 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#154 as decimal(12,2)) AS agg5#38, cast(c_birth_year#161 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#40] Input [11]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cd_dep_count#157, c_birth_year#161, ca_country#165, i_item_sk#167, i_item_id#29] -(99) HashAggregate [codegen id : 23] +(99) HashAggregate [codegen id : 5] Input [9]: [i_item_id#29, ca_country#165, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] Keys [2]: [i_item_id#29, ca_country#165] Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] @@ -623,10 +623,10 @@ Results [16]: [i_item_id#29, ca_country#165, sum#182, count#183, sum#184, count# Input [16]: [i_item_id#29, ca_country#165, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195] Arguments: hashpartitioning(i_item_id#29, ca_country#165, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(101) CometColumnarToRow [codegen id : 24] +(101) CometNativeColumnarToRow Input [16]: [i_item_id#29, ca_country#165, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195] -(102) HashAggregate [codegen id : 24] +(102) HashAggregate [codegen id : 6] Input [16]: [i_item_id#29, ca_country#165, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195] Keys [2]: [i_item_id#29, ca_country#165] Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] @@ -641,49 +641,49 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#220), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(104) ColumnarToRow [codegen id : 31] +(104) ColumnarToRow [codegen id : 7] Input [9]: [cs_bill_customer_sk#212, cs_bill_cdemo_sk#213, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220] -(105) Filter [codegen id : 31] +(105) Filter [codegen id : 7] Input [9]: [cs_bill_customer_sk#212, cs_bill_cdemo_sk#213, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220] Condition : ((isnotnull(cs_bill_cdemo_sk#213) AND isnotnull(cs_bill_customer_sk#212)) AND isnotnull(cs_item_sk#214)) (106) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#221, cd_dep_count#222] -(107) BroadcastHashJoin [codegen id : 31] +(107) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_bill_cdemo_sk#213] Right keys [1]: [cd_demo_sk#221] Join type: Inner Join condition: None -(108) Project [codegen id : 31] +(108) Project [codegen id : 7] Output [9]: [cs_bill_customer_sk#212, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222] Input [11]: [cs_bill_customer_sk#212, cs_bill_cdemo_sk#213, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_demo_sk#221, cd_dep_count#222] (109) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#223, c_current_cdemo_sk#224, c_current_addr_sk#225, c_birth_year#226] -(110) BroadcastHashJoin [codegen id : 31] +(110) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_bill_customer_sk#212] Right keys [1]: [c_customer_sk#223] Join type: Inner Join condition: None -(111) Project [codegen id : 31] +(111) Project [codegen id : 7] Output [11]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_current_cdemo_sk#224, c_current_addr_sk#225, c_birth_year#226] Input [13]: [cs_bill_customer_sk#212, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_customer_sk#223, c_current_cdemo_sk#224, c_current_addr_sk#225, c_birth_year#226] (112) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#227] -(113) BroadcastHashJoin [codegen id : 31] +(113) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_current_cdemo_sk#224] Right keys [1]: [cd_demo_sk#227] Join type: Inner Join condition: None -(114) Project [codegen id : 31] +(114) Project [codegen id : 7] Output [10]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_current_addr_sk#225, c_birth_year#226] Input [12]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_current_cdemo_sk#224, c_current_addr_sk#225, c_birth_year#226, cd_demo_sk#227] @@ -702,50 +702,50 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [2]: [ca_address_sk#228, ca_state#229] Arguments: [ca_address_sk#228], [ca_address_sk#228] -(118) CometColumnarToRow [codegen id : 28] +(118) CometNativeColumnarToRow Input [1]: [ca_address_sk#228] (119) BroadcastExchange Input [1]: [ca_address_sk#228] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(120) BroadcastHashJoin [codegen id : 31] +(120) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_current_addr_sk#225] Right keys [1]: [ca_address_sk#228] Join type: Inner Join condition: None -(121) Project [codegen id : 31] +(121) Project [codegen id : 7] Output [9]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_birth_year#226] Input [11]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_current_addr_sk#225, c_birth_year#226, ca_address_sk#228] (122) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#230] -(123) BroadcastHashJoin [codegen id : 31] +(123) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#220] Right keys [1]: [d_date_sk#230] Join type: Inner Join condition: None -(124) Project [codegen id : 31] +(124) Project [codegen id : 7] Output [8]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cd_dep_count#222, c_birth_year#226] Input [10]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_birth_year#226, d_date_sk#230] (125) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#231, i_item_id#29] -(126) BroadcastHashJoin [codegen id : 31] +(126) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_item_sk#214] Right keys [1]: [i_item_sk#231] Join type: Inner Join condition: None -(127) Project [codegen id : 31] +(127) Project [codegen id : 7] Output [8]: [i_item_id#29, cast(cs_quantity#215 as decimal(12,2)) AS agg1#34, cast(cs_list_price#216 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#218 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#217 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#219 as decimal(12,2)) AS agg5#38, cast(c_birth_year#226 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#222 as decimal(12,2)) AS agg7#40] Input [10]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cd_dep_count#222, c_birth_year#226, i_item_sk#231, i_item_id#29] -(128) HashAggregate [codegen id : 31] +(128) HashAggregate [codegen id : 7] Input [8]: [i_item_id#29, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] Keys [1]: [i_item_id#29] Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] @@ -756,10 +756,10 @@ Results [15]: [i_item_id#29, sum#246, count#247, sum#248, count#249, sum#250, co Input [15]: [i_item_id#29, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259] Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(130) CometColumnarToRow [codegen id : 32] +(130) CometNativeColumnarToRow Input [15]: [i_item_id#29, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259] -(131) HashAggregate [codegen id : 32] +(131) HashAggregate [codegen id : 8] Input [15]: [i_item_id#29, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259] Keys [1]: [i_item_id#29] Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] @@ -774,75 +774,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#285), dynamicpruningexpression(cs_s PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(133) ColumnarToRow [codegen id : 39] +(133) ColumnarToRow [codegen id : 9] Input [9]: [cs_bill_customer_sk#277, cs_bill_cdemo_sk#278, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285] -(134) Filter [codegen id : 39] +(134) Filter [codegen id : 9] Input [9]: [cs_bill_customer_sk#277, cs_bill_cdemo_sk#278, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285] Condition : ((isnotnull(cs_bill_cdemo_sk#278) AND isnotnull(cs_bill_customer_sk#277)) AND isnotnull(cs_item_sk#279)) (135) ReusedExchange [Reuses operator id: 8] Output [2]: [cd_demo_sk#286, cd_dep_count#287] -(136) BroadcastHashJoin [codegen id : 39] +(136) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_bill_cdemo_sk#278] Right keys [1]: [cd_demo_sk#286] Join type: Inner Join condition: None -(137) Project [codegen id : 39] +(137) Project [codegen id : 9] Output [9]: [cs_bill_customer_sk#277, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287] Input [11]: [cs_bill_customer_sk#277, cs_bill_cdemo_sk#278, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_demo_sk#286, cd_dep_count#287] (138) ReusedExchange [Reuses operator id: 15] Output [4]: [c_customer_sk#288, c_current_cdemo_sk#289, c_current_addr_sk#290, c_birth_year#291] -(139) BroadcastHashJoin [codegen id : 39] +(139) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_bill_customer_sk#277] Right keys [1]: [c_customer_sk#288] Join type: Inner Join condition: None -(140) Project [codegen id : 39] +(140) Project [codegen id : 9] Output [11]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_current_cdemo_sk#289, c_current_addr_sk#290, c_birth_year#291] Input [13]: [cs_bill_customer_sk#277, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_customer_sk#288, c_current_cdemo_sk#289, c_current_addr_sk#290, c_birth_year#291] (141) ReusedExchange [Reuses operator id: 21] Output [1]: [cd_demo_sk#292] -(142) BroadcastHashJoin [codegen id : 39] +(142) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#289] Right keys [1]: [cd_demo_sk#292] Join type: Inner Join condition: None -(143) Project [codegen id : 39] +(143) Project [codegen id : 9] Output [10]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_current_addr_sk#290, c_birth_year#291] Input [12]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_current_cdemo_sk#289, c_current_addr_sk#290, c_birth_year#291, cd_demo_sk#292] (144) ReusedExchange [Reuses operator id: 119] Output [1]: [ca_address_sk#293] -(145) BroadcastHashJoin [codegen id : 39] +(145) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#290] Right keys [1]: [ca_address_sk#293] Join type: Inner Join condition: None -(146) Project [codegen id : 39] +(146) Project [codegen id : 9] Output [9]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_birth_year#291] Input [11]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_current_addr_sk#290, c_birth_year#291, ca_address_sk#293] (147) ReusedExchange [Reuses operator id: 166] Output [1]: [d_date_sk#294] -(148) BroadcastHashJoin [codegen id : 39] +(148) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#285] Right keys [1]: [d_date_sk#294] Join type: Inner Join condition: None -(149) Project [codegen id : 39] +(149) Project [codegen id : 9] Output [8]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cd_dep_count#287, c_birth_year#291] Input [10]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_birth_year#291, d_date_sk#294] @@ -857,24 +857,24 @@ ReadSchema: struct Input [1]: [i_item_sk#295] Condition : isnotnull(i_item_sk#295) -(152) CometColumnarToRow [codegen id : 38] +(152) CometNativeColumnarToRow Input [1]: [i_item_sk#295] (153) BroadcastExchange Input [1]: [i_item_sk#295] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] -(154) BroadcastHashJoin [codegen id : 39] +(154) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_item_sk#279] Right keys [1]: [i_item_sk#295] Join type: Inner Join condition: None -(155) Project [codegen id : 39] +(155) Project [codegen id : 9] Output [7]: [cast(cs_quantity#280 as decimal(12,2)) AS agg1#34, cast(cs_list_price#281 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#283 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#282 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#284 as decimal(12,2)) AS agg5#38, cast(c_birth_year#291 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#287 as decimal(12,2)) AS agg7#40] Input [9]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cd_dep_count#287, c_birth_year#291, i_item_sk#295] -(156) HashAggregate [codegen id : 39] +(156) HashAggregate [codegen id : 9] Input [7]: [agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] Keys: [] Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] @@ -885,10 +885,10 @@ Results [14]: [sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#3 Input [14]: [sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321, sum#322, count#323] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(158) CometColumnarToRow [codegen id : 40] +(158) CometNativeColumnarToRow Input [14]: [sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321, sum#322, count#323] -(159) HashAggregate [codegen id : 40] +(159) HashAggregate [codegen id : 10] Input [14]: [sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321, sum#322, count#323] Keys: [] Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] @@ -905,7 +905,7 @@ Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_ Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (166) -+- * CometColumnarToRow (165) ++- CometNativeColumnarToRow (165) +- CometProject (164) +- CometFilter (163) +- CometNativeScan parquet spark_catalog.default.date_dim (162) @@ -926,7 +926,7 @@ Condition : ((isnotnull(d_year#342) AND (d_year#342 = 2001)) AND isnotnull(d_dat Input [2]: [d_date_sk#26, d_year#342] Arguments: [d_date_sk#26], [d_date_sk#26] -(165) CometColumnarToRow [codegen id : 1] +(165) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (166) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt index eb411c16e8..8ad4e890ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,41 +21,41 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -75,36 +75,36 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -124,36 +124,36 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -173,36 +173,36 @@ TakeOrderedAndProject : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -222,31 +222,31 @@ TakeOrderedAndProject : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/simplified.txt index 71745258b5..a123d3d8ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] Union - WholeStageCodegen (8) + WholeStageCodegen (2) HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) + WholeStageCodegen (1) HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -25,59 +25,47 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeColumnarToRow + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] InputAdapter BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeColumnarToRow + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] InputAdapter BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + CometNativeColumnarToRow + CometFilter [cd_demo_sk] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] + CometFilter [ca_address_sk,ca_county,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) + WholeStageCodegen (3) HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -104,22 +92,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #9 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) + WholeStageCodegen (6) HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) + WholeStageCodegen (5) HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -146,22 +132,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #11 - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + CometNativeColumnarToRow + CometProject [ca_address_sk,ca_country] + CometFilter [ca_address_sk,ca_state,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) + WholeStageCodegen (8) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #12 - WholeStageCodegen (31) + WholeStageCodegen (7) HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -188,22 +172,20 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [cd_demo_sk] #5 InputAdapter BroadcastExchange #13 - WholeStageCodegen (28) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) + WholeStageCodegen (10) HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #14 - WholeStageCodegen (39) + WholeStageCodegen (9) HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -234,8 +216,6 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #15 - WholeStageCodegen (38) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/explain.txt index 86072fd830..9f267a215d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/explain.txt @@ -807,7 +807,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, a Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (151) -+- * CometColumnarToRow (150) ++- CometNativeColumnarToRow (150) +- CometProject (149) +- CometFilter (148) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) @@ -828,7 +828,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(150) CometColumnarToRow [codegen id : 1] +(150) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (151) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt index b18a444bb9..148f6d4a6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/simplified.txt index 1572a2a240..ce665c5294 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt index 86072fd830..9f267a215d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt @@ -807,7 +807,7 @@ Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, a Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (151) -+- * CometColumnarToRow (150) ++- CometNativeColumnarToRow (150) +- CometProject (149) +- CometFilter (148) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) @@ -828,7 +828,7 @@ Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#26, d_year#27] Arguments: [d_date_sk#26], [d_date_sk#26] -(150) CometColumnarToRow [codegen id : 1] +(150) CometNativeColumnarToRow Input [1]: [d_date_sk#26] (151) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/extended.txt index b18a444bb9..148f6d4a6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/extended.txt @@ -20,7 +20,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 1572a2a240..ce665c5294 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 @@ -22,12 +22,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 CometProject [cd_demo_sk,cd_dep_count] CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/explain.txt index 5a32d4019b..460a888704 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/explain.txt @@ -2,11 +2,11 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -17,7 +17,7 @@ TakeOrderedAndProject (23) : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.catalog_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -32,10 +32,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] Condition : isnotnull(cs_item_sk#1) @@ -54,37 +54,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] @@ -95,10 +95,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] @@ -113,14 +113,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] @@ -132,7 +132,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometNativeScan parquet spark_catalog.default.date_dim (24) @@ -153,7 +153,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt index 369ec68bb4..6634edcbed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -17,17 +17,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/simplified.txt index 7bc0779e53..0c80fb3016 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/simplified.txt @@ -1,43 +1,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) + WholeStageCodegen (3) Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] InputAdapter Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt index 322bbe63b3..fb3e9cf73a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt index cf18e68a3d..7d1b101ff1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index 322bbe63b3..fb3e9cf73a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -133,7 +133,7 @@ Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_it Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * CometColumnarToRow (27) ++- CometNativeColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(27) CometColumnarToRow [codegen id : 1] +(27) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (28) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt index cd52b2cd12..0cfd81f36f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 cf18e68a3d..7d1b101ff1 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 @@ -19,12 +19,10 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q22.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/explain.txt index aa06ae2c5f..e098401edc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (24) +- * HashAggregate (23) - +- * CometColumnarToRow (22) + +- CometNativeColumnarToRow (22) +- CometColumnarExchange (21) +- * HashAggregate (20) +- * Expand (19) @@ -16,12 +16,12 @@ TakeOrderedAndProject (24) : : : +- Scan parquet spark_catalog.default.inventory (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) +- BroadcastExchange (16) - +- * CometColumnarToRow (15) + +- CometNativeColumnarToRow (15) +- CometNativeScan parquet spark_catalog.default.warehouse (14) @@ -33,23 +33,23 @@ PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] Condition : isnotnull(inv_item_sk#1) (4) ReusedExchange [Reuses operator id: 29] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] @@ -68,20 +68,20 @@ Condition : isnotnull(i_item_sk#6) Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#7, 50, true, false, true) AS i_brand#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#10, 50, true, false, true) AS i_product_name#14] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] (11) BroadcastExchange Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] @@ -91,26 +91,26 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(15) CometColumnarToRow [codegen id : 3] +(15) CometNativeColumnarToRow Input: [] (16) BroadcastExchange Input: [] Arguments: IdentityBroadcastMode, [plan_id=2] -(17) BroadcastNestedLoopJoin [codegen id : 4] +(17) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -(19) Expand [codegen id : 4] +(19) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] @@ -121,10 +121,10 @@ Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_gr Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] Functions [1]: [avg(inv_quantity_on_hand#2)] @@ -139,7 +139,7 @@ Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (29) -+- * CometColumnarToRow (28) ++- CometNativeColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometNativeScan parquet spark_catalog.default.date_dim (25) @@ -160,7 +160,7 @@ Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#26] Arguments: [d_date_sk#5], [d_date_sk#5] -(28) CometColumnarToRow [codegen id : 1] +(28) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (29) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt index fb0e21025e..108e3c13f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Expand @@ -15,22 +15,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/simplified.txt index 457dff46e2..954f5cb42e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,25 +19,19 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometNativeScan parquet spark_catalog.default.warehouse + CometNativeColumnarToRow + CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/explain.txt index f85c898208..daa4c97503 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/explain.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject (26) : +- CometFilter (10) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) @@ -95,7 +95,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(15) CometColumnarToRow [codegen id : 2] +(15) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse @@ -104,26 +104,26 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(17) CometColumnarToRow [codegen id : 1] +(17) CometNativeColumnarToRow Input: [] (18) BroadcastExchange Input: [] Arguments: IdentityBroadcastMode, [plan_id=1] -(19) BroadcastNestedLoopJoin [codegen id : 2] +(19) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(20) Project [codegen id : 2] +(20) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(21) Expand [codegen id : 2] +(21) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] @@ -134,10 +134,10 @@ Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_gr Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(24) CometColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(25) HashAggregate [codegen id : 3] +(25) HashAggregate [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [avg(inv_quantity_on_hand#2)] @@ -152,7 +152,7 @@ Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt index 6cb89465d2..5da4848ebe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -28,7 +28,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/simplified.txt index 221c6063ce..11d581a487 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,12 +19,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -35,7 +33,5 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + CometNativeColumnarToRow + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt index f85c898208..daa4c97503 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt @@ -23,7 +23,7 @@ TakeOrderedAndProject (26) : +- CometFilter (10) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) +- BroadcastExchange (18) - +- * CometColumnarToRow (17) + +- CometNativeColumnarToRow (17) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) @@ -95,7 +95,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(15) CometColumnarToRow [codegen id : 2] +(15) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] (16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse @@ -104,26 +104,26 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(17) CometColumnarToRow [codegen id : 1] +(17) CometNativeColumnarToRow Input: [] (18) BroadcastExchange Input: [] Arguments: IdentityBroadcastMode, [plan_id=1] -(19) BroadcastNestedLoopJoin [codegen id : 2] +(19) BroadcastNestedLoopJoin [codegen id : 1] Join type: Inner Join condition: None -(20) Project [codegen id : 2] +(20) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] -(21) Expand [codegen id : 2] +(21) Expand [codegen id : 1] Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -(22) HashAggregate [codegen id : 2] +(22) HashAggregate [codegen id : 1] Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [partial_avg(inv_quantity_on_hand#2)] @@ -134,10 +134,10 @@ Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_gr Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(24) CometColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -(25) HashAggregate [codegen id : 3] +(25) HashAggregate [codegen id : 2] Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] Functions [1]: [avg(inv_quantity_on_hand#2)] @@ -152,7 +152,7 @@ Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 BroadcastExchange (31) -+- * CometColumnarToRow (30) ++- CometNativeColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(30) CometColumnarToRow [codegen id : 1] +(30) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (31) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt index 6cb89465d2..5da4848ebe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -28,7 +28,7 @@ TakeOrderedAndProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt index 221c6063ce..11d581a487 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] CometColumnarToRow InputAdapter CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] @@ -19,12 +19,10 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -35,7 +33,5 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + CometNativeColumnarToRow + CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/explain.txt index e53305a55b..494ec37d8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/explain.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject (55) :- * HashAggregate (25) : +- * HashAggregate (24) : +- * HashAggregate (23) - : +- * CometColumnarToRow (22) + : +- CometNativeColumnarToRow (22) : +- CometColumnarExchange (21) : +- * HashAggregate (20) : +- * Project (19) @@ -18,41 +18,41 @@ TakeOrderedAndProject (55) : : : : +- Scan parquet spark_catalog.default.inventory (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.item (7) : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.warehouse (14) :- * HashAggregate (32) - : +- * CometColumnarToRow (31) + : +- CometNativeColumnarToRow (31) : +- CometColumnarExchange (30) : +- * HashAggregate (29) : +- * HashAggregate (28) - : +- * CometColumnarToRow (27) + : +- CometNativeColumnarToRow (27) : +- ReusedExchange (26) :- * HashAggregate (39) - : +- * CometColumnarToRow (38) + : +- CometNativeColumnarToRow (38) : +- CometColumnarExchange (37) : +- * HashAggregate (36) : +- * HashAggregate (35) - : +- * CometColumnarToRow (34) + : +- CometNativeColumnarToRow (34) : +- ReusedExchange (33) :- * HashAggregate (46) - : +- * CometColumnarToRow (45) + : +- CometNativeColumnarToRow (45) : +- CometColumnarExchange (44) : +- * HashAggregate (43) : +- * HashAggregate (42) - : +- * CometColumnarToRow (41) + : +- CometNativeColumnarToRow (41) : +- ReusedExchange (40) +- * HashAggregate (53) - +- * CometColumnarToRow (52) + +- CometNativeColumnarToRow (52) +- CometColumnarExchange (51) +- * HashAggregate (50) +- * HashAggregate (49) - +- * CometColumnarToRow (48) + +- CometNativeColumnarToRow (48) +- ReusedExchange (47) @@ -64,23 +64,23 @@ PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_s PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (4) ReusedExchange [Reuses operator id: 60] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] @@ -99,20 +99,20 @@ Condition : isnotnull(i_item_sk#7) Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] (11) BroadcastExchange Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] @@ -127,24 +127,24 @@ ReadSchema: struct Input [1]: [w_warehouse_sk#16] Condition : isnotnull(w_warehouse_sk#16) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [1]: [w_warehouse_sk#16] (17) BroadcastExchange Input [1]: [w_warehouse_sk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#2] Right keys [1]: [w_warehouse_sk#16] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [partial_avg(inv_quantity_on_hand#3)] @@ -155,24 +155,24 @@ Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 2] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#3)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] Results [5]: [i_product_name#15 AS i_product_name#22, i_brand#12 AS i_brand#23, i_class#13 AS i_class#24, i_category#14 AS i_category#25, avg(inv_quantity_on_hand#3)#21 AS qoh#26] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#26] Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] Functions [1]: [partial_avg(qoh#26)] Aggregate Attributes [2]: [sum#27, count#28] Results [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] -(25) HashAggregate [codegen id : 5] +(25) HashAggregate [codegen id : 2] Input [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] Functions [1]: [avg(qoh#26)] @@ -182,17 +182,17 @@ Results [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, avg(qoh# (26) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] -(27) CometColumnarToRow [codegen id : 10] +(27) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] -(28) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 4] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#35)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#21] Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(inv_quantity_on_hand#35)#21 AS qoh#26] -(29) HashAggregate [codegen id : 10] +(29) HashAggregate [codegen id : 4] Input [4]: [i_product_name#15, i_brand#12, i_class#13, qoh#26] Keys [3]: [i_product_name#15, i_brand#12, i_class#13] Functions [1]: [partial_avg(qoh#26)] @@ -203,10 +203,10 @@ Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#38, count#39] Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#38, count#39] Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(31) CometColumnarToRow [codegen id : 11] +(31) CometNativeColumnarToRow Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#38, count#39] -(32) HashAggregate [codegen id : 11] +(32) HashAggregate [codegen id : 5] Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#38, count#39] Keys [3]: [i_product_name#15, i_brand#12, i_class#13] Functions [1]: [avg(qoh#26)] @@ -216,17 +216,17 @@ Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#41, (33) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#43, count#44] -(34) CometColumnarToRow [codegen id : 16] +(34) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#43, count#44] -(35) HashAggregate [codegen id : 16] +(35) HashAggregate [codegen id : 7] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#43, count#44] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#45)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#45)#21] Results [3]: [i_product_name#15, i_brand#12, avg(inv_quantity_on_hand#45)#21 AS qoh#26] -(36) HashAggregate [codegen id : 16] +(36) HashAggregate [codegen id : 7] Input [3]: [i_product_name#15, i_brand#12, qoh#26] Keys [2]: [i_product_name#15, i_brand#12] Functions [1]: [partial_avg(qoh#26)] @@ -237,10 +237,10 @@ Results [4]: [i_product_name#15, i_brand#12, sum#48, count#49] Input [4]: [i_product_name#15, i_brand#12, sum#48, count#49] Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(38) CometColumnarToRow [codegen id : 17] +(38) CometNativeColumnarToRow Input [4]: [i_product_name#15, i_brand#12, sum#48, count#49] -(39) HashAggregate [codegen id : 17] +(39) HashAggregate [codegen id : 8] Input [4]: [i_product_name#15, i_brand#12, sum#48, count#49] Keys [2]: [i_product_name#15, i_brand#12] Functions [1]: [avg(qoh#26)] @@ -250,17 +250,17 @@ Results [5]: [i_product_name#15, i_brand#12, null AS i_class#51, null AS i_categ (40) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#54, count#55] -(41) CometColumnarToRow [codegen id : 22] +(41) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#54, count#55] -(42) HashAggregate [codegen id : 22] +(42) HashAggregate [codegen id : 10] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#54, count#55] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#56)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#56)#21] Results [2]: [i_product_name#15, avg(inv_quantity_on_hand#56)#21 AS qoh#26] -(43) HashAggregate [codegen id : 22] +(43) HashAggregate [codegen id : 10] Input [2]: [i_product_name#15, qoh#26] Keys [1]: [i_product_name#15] Functions [1]: [partial_avg(qoh#26)] @@ -271,10 +271,10 @@ Results [3]: [i_product_name#15, sum#59, count#60] Input [3]: [i_product_name#15, sum#59, count#60] Arguments: hashpartitioning(i_product_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(45) CometColumnarToRow [codegen id : 23] +(45) CometNativeColumnarToRow Input [3]: [i_product_name#15, sum#59, count#60] -(46) HashAggregate [codegen id : 23] +(46) HashAggregate [codegen id : 11] Input [3]: [i_product_name#15, sum#59, count#60] Keys [1]: [i_product_name#15] Functions [1]: [avg(qoh#26)] @@ -284,17 +284,17 @@ Results [5]: [i_product_name#15, null AS i_brand#62, null AS i_class#63, null AS (47) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#66, count#67] -(48) CometColumnarToRow [codegen id : 28] +(48) CometNativeColumnarToRow Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#66, count#67] -(49) HashAggregate [codegen id : 28] +(49) HashAggregate [codegen id : 13] Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#66, count#67] Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] Functions [1]: [avg(inv_quantity_on_hand#68)] Aggregate Attributes [1]: [avg(inv_quantity_on_hand#68)#21] Results [1]: [avg(inv_quantity_on_hand#68)#21 AS qoh#26] -(50) HashAggregate [codegen id : 28] +(50) HashAggregate [codegen id : 13] Input [1]: [qoh#26] Keys: [] Functions [1]: [partial_avg(qoh#26)] @@ -305,10 +305,10 @@ Results [2]: [sum#71, count#72] Input [2]: [sum#71, count#72] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(52) CometColumnarToRow [codegen id : 29] +(52) CometNativeColumnarToRow Input [2]: [sum#71, count#72] -(53) HashAggregate [codegen id : 29] +(53) HashAggregate [codegen id : 14] Input [2]: [sum#71, count#72] Keys: [] Functions [1]: [avg(qoh#26)] @@ -325,7 +325,7 @@ Arguments: 100, [qoh#32 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (60) -+- * CometColumnarToRow (59) ++- CometNativeColumnarToRow (59) +- CometProject (58) +- CometFilter (57) +- CometNativeScan parquet spark_catalog.default.date_dim (56) @@ -346,7 +346,7 @@ Condition : (((isnotnull(d_month_seq#79) AND (d_month_seq#79 >= 1212)) AND (d_mo Input [2]: [d_date_sk#6, d_month_seq#79] Arguments: [d_date_sk#6], [d_date_sk#6] -(59) CometColumnarToRow [codegen id : 1] +(59) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (60) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt index 543f666a74..eaa5438d91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt @@ -3,7 +3,7 @@ TakeOrderedAndProject :- HashAggregate : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -17,30 +17,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -54,30 +54,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -91,30 +91,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -128,30 +128,30 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.warehouse +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -165,22 +165,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/simplified.txt index a55feaac8c..4f9ec6fb7a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/simplified.txt @@ -1,13 +1,13 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union - WholeStageCodegen (5) + WholeStageCodegen (2) HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) + WholeStageCodegen (1) HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] @@ -21,70 +21,64 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) + CometNativeColumnarToRow + CometFilter [w_warehouse_sk] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + WholeStageCodegen (5) HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) + WholeStageCodegen (4) HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) + WholeStageCodegen (8) HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) + WholeStageCodegen (7) HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) + WholeStageCodegen (11) HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_product_name] #7 - WholeStageCodegen (22) + WholeStageCodegen (10) HashAggregate [i_product_name,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) + WholeStageCodegen (14) HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #8 - WholeStageCodegen (28) + WholeStageCodegen (13) HashAggregate [qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow 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/q22a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/explain.txt index 189a54b584..4acc5c7de5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/explain.txt @@ -270,7 +270,7 @@ Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt index c018673888..4ccef655b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/simplified.txt index b59605103e..e6db35bb18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index 189a54b584..4acc5c7de5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -270,7 +270,7 @@ Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/extended.txt index c018673888..4ccef655b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -47,7 +47,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -78,7 +78,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 b59605103e..e6db35bb18 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #3 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/explain.txt index 6af787c781..4a5f260b1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/explain.txt @@ -1,19 +1,19 @@ == Physical Plan == -* CometColumnarToRow (50) +CometNativeColumnarToRow (50) +- CometSort (49) +- CometColumnarExchange (48) +- * Filter (47) +- * HashAggregate (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometColumnarExchange (44) +- * HashAggregate (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) + :- CometNativeColumnarToRow (31) : +- CometProject (30) : +- CometBroadcastHashJoin (29) : :- CometProject (24) @@ -45,7 +45,7 @@ : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_address (32) @@ -190,7 +190,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 2] +(31) CometNativeColumnarToRow Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] (32) CometNativeScan parquet spark_catalog.default.customer_address @@ -208,24 +208,24 @@ Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isno Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] (36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -236,17 +236,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometNativeColumnarToRow Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] @@ -257,17 +257,17 @@ Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty# Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometNativeColumnarToRow Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) @@ -279,23 +279,23 @@ Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(50) CometNativeColumnarToRow Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] * HashAggregate (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometColumnarExchange (78) +- * HashAggregate (77) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- CometColumnarExchange (74) +- * HashAggregate (73) +- * Project (72) +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) + :- CometNativeColumnarToRow (69) : +- CometProject (68) : +- CometBroadcastHashJoin (67) : :- CometProject (65) @@ -392,23 +392,23 @@ Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(69) CometColumnarToRow [codegen id : 2] +(69) CometNativeColumnarToRow Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] (70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(71) BroadcastHashJoin [codegen id : 2] +(71) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(72) Project [codegen id : 1] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(73) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] @@ -419,17 +419,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometColumnarToRow [codegen id : 3] +(75) CometNativeColumnarToRow Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(76) HashAggregate [codegen id : 3] +(76) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#42] -(77) HashAggregate [codegen id : 3] +(77) HashAggregate [codegen id : 2] Input [1]: [netpaid#42] Keys: [] Functions [1]: [partial_avg(netpaid#42)] @@ -440,10 +440,10 @@ Results [2]: [sum#75, count#76] Input [2]: [sum#75, count#76] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 4] +(79) CometNativeColumnarToRow Input [2]: [sum#75, count#76] -(80) HashAggregate [codegen id : 4] +(80) HashAggregate [codegen id : 3] Input [2]: [sum#75, count#76] Keys: [] Functions [1]: [avg(netpaid#42)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt index 53a4c64a61..4c22965f0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt @@ -1,19 +1,19 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Filter : +- Subquery : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometProject @@ -45,21 +45,21 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometProject @@ -91,7 +91,7 @@ CometColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/simplified.txt index b24e51723a..4758db93c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/simplified.txt @@ -1,64 +1,26 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] +CometNativeColumnarToRow + CometSort [c_last_name,c_first_name,s_store_name,paid] + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (3) + Filter [paid] + Subquery #1 + WholeStageCodegen (3) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] @@ -68,32 +30,66 @@ WholeStageCodegen (5) CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + ReusedExchange [sr_item_sk,sr_ticket_number] #5 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + WholeStageCodegen (2) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + WholeStageCodegen (1) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + InputAdapter + CometNativeColumnarToRow + CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] + CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometExchange [ss_ticket_number,ss_item_sk] #4 + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + CometSort [sr_item_sk,sr_ticket_number] + CometExchange [sr_ticket_number,sr_item_sk] #5 + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 + CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #9 + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/explain.txt index 02873e35d9..1a1dd7e0a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/explain.txt @@ -45,7 +45,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -190,7 +190,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -208,24 +208,24 @@ Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isno Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] (36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -236,17 +236,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] @@ -257,17 +257,17 @@ Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty# Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) @@ -279,7 +279,7 @@ Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(50) CometColumnarToRow [codegen id : 4] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== @@ -392,23 +392,23 @@ Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(69) CometColumnarToRow [codegen id : 2] +(69) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] (70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(71) BroadcastHashJoin [codegen id : 2] +(71) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(72) Project [codegen id : 1] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(73) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] @@ -419,17 +419,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometColumnarToRow [codegen id : 3] +(75) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(76) HashAggregate [codegen id : 3] +(76) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#42] -(77) HashAggregate [codegen id : 3] +(77) HashAggregate [codegen id : 2] Input [1]: [netpaid#42] Keys: [] Functions [1]: [partial_avg(netpaid#42)] @@ -440,10 +440,10 @@ Results [2]: [sum#75, count#76] Input [2]: [sum#75, count#76] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 4] +(79) CometColumnarToRow [codegen id : 3] Input [2]: [sum#75, count#76] -(80) HashAggregate [codegen id : 4] +(80) HashAggregate [codegen id : 3] Input [2]: [sum#75, count#76] Keys: [] Functions [1]: [avg(netpaid#42)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt index b9384ca04f..91b709aa47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt @@ -45,7 +45,7 @@ CometColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -91,7 +91,7 @@ CometColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/simplified.txt index 62f492f632..b657d3dae1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (5) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -47,13 +47,13 @@ WholeStageCodegen (5) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -91,9 +91,7 @@ WholeStageCodegen (5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt index 02873e35d9..1a1dd7e0a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt @@ -45,7 +45,7 @@ : +- CometFilter (26) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) @@ -190,7 +190,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -(31) CometColumnarToRow [codegen id : 2] +(31) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] (32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -208,24 +208,24 @@ Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isno Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] -(35) CometColumnarToRow [codegen id : 1] +(35) CometNativeColumnarToRow Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] (36) BroadcastExchange Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] -(37) BroadcastHashJoin [codegen id : 2] +(37) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] Join type: Inner Join condition: None -(38) Project [codegen id : 2] +(38) Project [codegen id : 1] Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -(39) HashAggregate [codegen id : 2] +(39) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] @@ -236,17 +236,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 3] +(41) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -(42) HashAggregate [codegen id : 3] +(42) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] -(43) HashAggregate [codegen id : 3] +(43) HashAggregate [codegen id : 2] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [partial_sum(netpaid#42)] @@ -257,17 +257,17 @@ Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty# Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 4] +(45) CometColumnarToRow [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -(46) HashAggregate [codegen id : 4] +(46) HashAggregate [codegen id : 3] Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] Functions [1]: [sum(netpaid#42)] Aggregate Attributes [1]: [sum(netpaid#42)#47] Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] -(47) Filter [codegen id : 4] +(47) Filter [codegen id : 3] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) @@ -279,7 +279,7 @@ Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] -(50) CometColumnarToRow [codegen id : 5] +(50) CometColumnarToRow [codegen id : 4] Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] ===== Subqueries ===== @@ -392,23 +392,23 @@ Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -(69) CometColumnarToRow [codegen id : 2] +(69) CometColumnarToRow [codegen id : 1] Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] (70) ReusedExchange [Reuses operator id: 36] Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(71) BroadcastHashJoin [codegen id : 2] +(71) BroadcastHashJoin [codegen id : 1] Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] Join type: Inner Join condition: None -(72) Project [codegen id : 2] +(72) Project [codegen id : 1] Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] -(73) HashAggregate [codegen id : 2] +(73) HashAggregate [codegen id : 1] Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] @@ -419,17 +419,17 @@ Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_ Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(75) CometColumnarToRow [codegen id : 3] +(75) CometColumnarToRow [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -(76) HashAggregate [codegen id : 3] +(76) HashAggregate [codegen id : 2] Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#42] -(77) HashAggregate [codegen id : 3] +(77) HashAggregate [codegen id : 2] Input [1]: [netpaid#42] Keys: [] Functions [1]: [partial_avg(netpaid#42)] @@ -440,10 +440,10 @@ Results [2]: [sum#75, count#76] Input [2]: [sum#75, count#76] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(79) CometColumnarToRow [codegen id : 4] +(79) CometColumnarToRow [codegen id : 3] Input [2]: [sum#75, count#76] -(80) HashAggregate [codegen id : 4] +(80) HashAggregate [codegen id : 3] Input [2]: [sum#75, count#76] Keys: [] Functions [1]: [avg(netpaid#42)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt index b9384ca04f..91b709aa47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt @@ -45,7 +45,7 @@ CometColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address @@ -91,7 +91,7 @@ CometColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt index 62f492f632..b657d3dae1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt @@ -1,23 +1,23 @@ -WholeStageCodegen (5) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [paid] Subquery #1 - WholeStageCodegen (4) + WholeStageCodegen (3) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] CometColumnarToRow InputAdapter CometColumnarExchange #10 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [netpaid] [sum,count,sum,count] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -47,13 +47,13 @@ WholeStageCodegen (5) CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] CometColumnarToRow InputAdapter CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) + WholeStageCodegen (1) HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] @@ -91,9 +91,7 @@ WholeStageCodegen (5) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] InputAdapter BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] + CometNativeColumnarToRow + CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] + CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/explain.txt index 0723f5c6b2..746f1bb5c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (78) +- Union (77) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * Project (27) @@ -17,23 +17,23 @@ TakeOrderedAndProject (78) : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) : : : +- ReusedExchange (11) : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) + : : +- CometNativeColumnarToRow (17) : : +- CometProject (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.store (14) : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) + : +- CometNativeColumnarToRow (24) : +- CometProject (23) : +- CometFilter (22) : +- CometNativeScan parquet spark_catalog.default.item (21) :- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) @@ -50,13 +50,13 @@ TakeOrderedAndProject (78) : : : : +- ReusedExchange (35) : : : +- ReusedExchange (38) : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) + : : +- CometNativeColumnarToRow (44) : : +- CometProject (43) : : +- CometFilter (42) : : +- CometNativeScan parquet spark_catalog.default.store (41) : +- ReusedExchange (48) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- CometColumnarExchange (74) +- * HashAggregate (73) +- * Project (72) @@ -74,7 +74,7 @@ TakeOrderedAndProject (78) : : +- ReusedExchange (61) : +- ReusedExchange (64) +- BroadcastExchange (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometFilter (68) +- CometNativeScan parquet spark_catalog.default.item (67) @@ -87,10 +87,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 5] +(2) ColumnarToRow [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -(3) Filter [codegen id : 5] +(3) Filter [codegen id : 1] Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) @@ -109,33 +109,33 @@ Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcha Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Arguments: [cd_demo_sk#10], [cd_demo_sk#10] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [1]: [cd_demo_sk#10] (8) BroadcastExchange Input [1]: [cd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 5] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#10] Join type: Inner Join condition: None -(10) Project [codegen id : 5] +(10) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] (11) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 5] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 5] +(13) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] @@ -154,20 +154,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#15, s_state#16] Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [2]: [s_store_sk#15, s_state#17] (18) BroadcastExchange Input [2]: [s_store_sk#15, s_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#15] Join type: Inner Join condition: None -(20) Project [codegen id : 5] +(20) Project [codegen id : 1] Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] @@ -186,24 +186,24 @@ Condition : isnotnull(i_item_sk#18) Input [2]: [i_item_sk#18, i_item_id#19] Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] -(24) CometColumnarToRow [codegen id : 4] +(24) CometNativeColumnarToRow Input [2]: [i_item_sk#18, i_item_id#20] (25) BroadcastExchange Input [2]: [i_item_sk#18, i_item_id#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#18] Join type: Inner Join condition: None -(27) Project [codegen id : 5] +(27) Project [codegen id : 1] Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] -(28) HashAggregate [codegen id : 5] +(28) HashAggregate [codegen id : 1] Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] Keys [2]: [i_item_id#21, s_state#22] Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] @@ -214,10 +214,10 @@ Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 6] +(30) CometNativeColumnarToRow Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -(31) HashAggregate [codegen id : 6] +(31) HashAggregate [codegen id : 2] Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] Keys [2]: [i_item_id#21, s_state#22] Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] @@ -232,36 +232,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(33) ColumnarToRow [codegen id : 11] +(33) ColumnarToRow [codegen id : 3] Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -(34) Filter [codegen id : 11] +(34) Filter [codegen id : 3] Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) (35) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#60] -(36) BroadcastHashJoin [codegen id : 11] +(36) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_cdemo_sk#53] Right keys [1]: [cd_demo_sk#60] Join type: Inner Join condition: None -(37) Project [codegen id : 11] +(37) Project [codegen id : 3] Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#60] (38) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#61] -(39) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_sold_date_sk#59] Right keys [1]: [d_date_sk#61] Join type: Inner Join condition: None -(40) Project [codegen id : 11] +(40) Project [codegen id : 3] Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#61] @@ -280,37 +280,37 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#62, s_state#63] Arguments: [s_store_sk#62], [s_store_sk#62] -(44) CometColumnarToRow [codegen id : 9] +(44) CometNativeColumnarToRow Input [1]: [s_store_sk#62] (45) BroadcastExchange Input [1]: [s_store_sk#62] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(46) BroadcastHashJoin [codegen id : 11] +(46) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_store_sk#54] Right keys [1]: [s_store_sk#62] Join type: Inner Join condition: None -(47) Project [codegen id : 11] +(47) Project [codegen id : 3] Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#62] (48) ReusedExchange [Reuses operator id: 25] Output [2]: [i_item_sk#64, i_item_id#20] -(49) BroadcastHashJoin [codegen id : 11] +(49) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ss_item_sk#52] Right keys [1]: [i_item_sk#64] Join type: Inner Join condition: None -(50) Project [codegen id : 11] +(50) Project [codegen id : 3] Output [5]: [i_item_id#20, ss_quantity#55 AS agg1#23, ss_list_price#56 AS agg2#24, ss_coupon_amt#58 AS agg3#25, ss_sales_price#57 AS agg4#26] Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#64, i_item_id#20] -(51) HashAggregate [codegen id : 11] +(51) HashAggregate [codegen id : 3] Input [5]: [i_item_id#20, agg1#23, agg2#24, agg3#25, agg4#26] Keys [1]: [i_item_id#20] Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] @@ -321,10 +321,10 @@ Results [9]: [i_item_id#20, sum#73, count#74, sum#75, count#76, sum#77, count#78 Input [9]: [i_item_id#20, sum#73, count#74, sum#75, count#76, sum#77, count#78, sum#79, count#80] Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(53) CometColumnarToRow [codegen id : 12] +(53) CometNativeColumnarToRow Input [9]: [i_item_id#20, sum#73, count#74, sum#75, count#76, sum#77, count#78, sum#79, count#80] -(54) HashAggregate [codegen id : 12] +(54) HashAggregate [codegen id : 4] Input [9]: [i_item_id#20, sum#73, count#74, sum#75, count#76, sum#77, count#78, sum#79, count#80] Keys [1]: [i_item_id#20] Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] @@ -339,49 +339,49 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(56) ColumnarToRow [codegen id : 17] +(56) ColumnarToRow [codegen id : 5] Input [8]: [ss_item_sk#91, ss_cdemo_sk#92, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98] -(57) Filter [codegen id : 17] +(57) Filter [codegen id : 5] Input [8]: [ss_item_sk#91, ss_cdemo_sk#92, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98] Condition : ((isnotnull(ss_cdemo_sk#92) AND isnotnull(ss_store_sk#93)) AND isnotnull(ss_item_sk#91)) (58) ReusedExchange [Reuses operator id: 8] Output [1]: [cd_demo_sk#99] -(59) BroadcastHashJoin [codegen id : 17] +(59) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#92] Right keys [1]: [cd_demo_sk#99] Join type: Inner Join condition: None -(60) Project [codegen id : 17] +(60) Project [codegen id : 5] Output [7]: [ss_item_sk#91, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98] Input [9]: [ss_item_sk#91, ss_cdemo_sk#92, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98, cd_demo_sk#99] (61) ReusedExchange [Reuses operator id: 83] Output [1]: [d_date_sk#100] -(62) BroadcastHashJoin [codegen id : 17] +(62) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#98] Right keys [1]: [d_date_sk#100] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(63) Project [codegen id : 5] Output [6]: [ss_item_sk#91, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97] Input [8]: [ss_item_sk#91, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98, d_date_sk#100] (64) ReusedExchange [Reuses operator id: 45] Output [1]: [s_store_sk#101] -(65) BroadcastHashJoin [codegen id : 17] +(65) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#93] Right keys [1]: [s_store_sk#101] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(66) Project [codegen id : 5] Output [5]: [ss_item_sk#91, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97] Input [7]: [ss_item_sk#91, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, s_store_sk#101] @@ -396,24 +396,24 @@ ReadSchema: struct Input [1]: [i_item_sk#102] Condition : isnotnull(i_item_sk#102) -(69) CometColumnarToRow [codegen id : 16] +(69) CometNativeColumnarToRow Input [1]: [i_item_sk#102] (70) BroadcastExchange Input [1]: [i_item_sk#102] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(71) BroadcastHashJoin [codegen id : 17] +(71) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#91] Right keys [1]: [i_item_sk#102] Join type: Inner Join condition: None -(72) Project [codegen id : 17] +(72) Project [codegen id : 5] Output [4]: [ss_quantity#94 AS agg1#23, ss_list_price#95 AS agg2#24, ss_coupon_amt#97 AS agg3#25, ss_sales_price#96 AS agg4#26] Input [6]: [ss_item_sk#91, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, i_item_sk#102] -(73) HashAggregate [codegen id : 17] +(73) HashAggregate [codegen id : 5] Input [4]: [agg1#23, agg2#24, agg3#25, agg4#26] Keys: [] Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] @@ -424,10 +424,10 @@ Results [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#11 Input [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(75) CometColumnarToRow [codegen id : 18] +(75) CometNativeColumnarToRow Input [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] -(76) HashAggregate [codegen id : 18] +(76) HashAggregate [codegen id : 6] Input [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] Keys: [] Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] @@ -444,7 +444,7 @@ Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_i Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometNativeScan parquet spark_catalog.default.date_dim (79) @@ -465,7 +465,7 @@ Condition : ((isnotnull(d_year#130) AND (d_year#130 = 1998)) AND isnotnull(d_dat Input [2]: [d_date_sk#14, d_year#130] Arguments: [d_date_sk#14], [d_date_sk#14] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt index 9926d14ae2..4eec8b0953 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -17,32 +17,32 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,27 +58,27 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -94,22 +94,22 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/simplified.txt index c7093301af..2dad05ae3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Union - WholeStageCodegen (6) + WholeStageCodegen (2) HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id,s_state] #1 - WholeStageCodegen (5) + WholeStageCodegen (1) HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -21,44 +21,36 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] InputAdapter ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) + CometNativeColumnarToRow + CometProject [i_item_id] [i_item_sk,i_item_id] + CometFilter [i_item_sk,i_item_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (4) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (11) + WholeStageCodegen (3) HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -79,20 +71,18 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ReusedExchange [d_date_sk] #2 InputAdapter BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) + WholeStageCodegen (6) HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange #8 - WholeStageCodegen (17) + WholeStageCodegen (5) HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -115,8 +105,6 @@ TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] ReusedExchange [s_store_sk] #7 InputAdapter BroadcastExchange #9 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] + CometNativeColumnarToRow + CometFilter [i_item_sk] + CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/explain.txt index af8de10b38..92210ee952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/explain.txt @@ -402,7 +402,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -423,7 +423,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt index affbc9ef37..a3e6b17491 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/simplified.txt index 0d18ca5626..d24cb217a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/simplified.txt @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt index af8de10b38..92210ee952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt @@ -402,7 +402,7 @@ Input [7]: [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometProject (74) +- CometFilter (73) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) @@ -423,7 +423,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (76) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/extended.txt index affbc9ef37..a3e6b17491 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 0d18ca5626..d24cb217a9 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 @@ -18,12 +18,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + 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] CometBroadcastExchange [cd_demo_sk] #3 CometProject [cd_demo_sk] CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/explain.txt index 587f232ea8..954779ae6b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/explain.txt @@ -1,12 +1,12 @@ == Physical Plan == -* CometColumnarToRow (35) +CometNativeColumnarToRow (35) +- CometSort (34) +- CometColumnarExchange (33) +- * Project (32) +- * BroadcastHashJoin Inner BuildRight (31) :- * Filter (25) : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -20,17 +20,17 @@ : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) +- BroadcastExchange (30) - +- * CometColumnarToRow (29) + +- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.customer (26) @@ -44,23 +44,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) (4) ReusedExchange [Reuses operator id: 40] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] @@ -79,20 +79,20 @@ Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND is Input [2]: [s_store_sk#8, s_county#9] Arguments: [s_store_sk#8], [s_store_sk#8] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [1]: [s_store_sk#8] (11) BroadcastExchange Input [1]: [s_store_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#3] Right keys [1]: [s_store_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] @@ -111,24 +111,24 @@ Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apac Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] Arguments: [hd_demo_sk#10], [hd_demo_sk#10] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [hd_demo_sk#10] (18) BroadcastExchange Input [1]: [hd_demo_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#10] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [2]: [ss_customer_sk#1, ss_ticket_number#4] Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [2]: [ss_customer_sk#1, ss_ticket_number#4] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [partial_count(1)] @@ -139,17 +139,17 @@ Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 6] +(23) CometNativeColumnarToRow Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -(24) HashAggregate [codegen id : 6] +(24) HashAggregate [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#16] Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) @@ -168,20 +168,20 @@ Condition : isnotnull(c_customer_sk#18) Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] -(29) CometColumnarToRow [codegen id : 5] +(29) CometNativeColumnarToRow Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] (30) BroadcastExchange Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 6] +(31) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_customer_sk#1] Right keys [1]: [c_customer_sk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 6] +(32) Project [codegen id : 2] Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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] @@ -193,14 +193,14 @@ Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC 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: [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 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] -(35) CometColumnarToRow [codegen id : 7] +(35) CometNativeColumnarToRow Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (40) -+- * CometColumnarToRow (39) ++- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometNativeScan parquet spark_catalog.default.date_dim (36) @@ -221,7 +221,7 @@ Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d Input [3]: [d_date_sk#7, d_year#27, d_dom#28] Arguments: [d_date_sk#7], [d_date_sk#7] -(39) CometColumnarToRow [codegen id : 1] +(39) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (40) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt index 9e4596c12c..40230582bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt @@ -1,11 +1,11 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- BroadcastHashJoin :- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -19,27 +19,27 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.household_demographics +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/simplified.txt index 63c6ac3645..bf781c83d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/simplified.txt @@ -1,59 +1,49 @@ -WholeStageCodegen (7) - 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 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] +CometNativeColumnarToRow + 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 + WholeStageCodegen (2) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #3 + Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_county] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/explain.txt index 50af5f4f75..b114cc7dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/explain.txt @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/simplified.txt index d57afd4fcf..ee10f76856 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/simplified.txt @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 50af5f4f75..b114cc7dd5 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 @@ -191,7 +191,7 @@ Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_f Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (38) -+- * CometColumnarToRow (37) ++- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) @@ -212,7 +212,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(37) CometColumnarToRow [codegen id : 1] +(37) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (38) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/extended.txt index 3d9d23d2cc..65e639c1d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d57afd4fcf..ee10f76856 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 @@ -19,12 +19,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_dom] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/explain.txt index d3c5709bdd..4272245c06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (45) +- * HashAggregate (44) - +- * CometColumnarToRow (43) + +- CometNativeColumnarToRow (43) +- CometColumnarExchange (42) +- * HashAggregate (41) +- * Project (40) @@ -13,7 +13,7 @@ TakeOrderedAndProject (45) : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) + : : : : :- CometNativeColumnarToRow (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : : +- BroadcastExchange (9) @@ -35,12 +35,12 @@ TakeOrderedAndProject (45) : : : +- Scan parquet spark_catalog.default.catalog_sales (18) : : +- ReusedExchange (20) : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometProject (29) : +- CometFilter (28) : +- CometNativeScan parquet spark_catalog.default.customer_address (27) +- BroadcastExchange (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) @@ -57,7 +57,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] (6) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#7] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#6] Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] @@ -87,7 +87,7 @@ Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] Input [1]: [ss_customer_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ss_customer_sk#6] Join type: LeftSemi @@ -100,19 +100,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] (13) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#11] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#10] Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#9] @@ -120,7 +120,7 @@ Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#9] Input [1]: [ws_bill_customer_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(17) BroadcastHashJoin [codegen id : 9] +(17) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#10] Join type: ExistenceJoin(exists#2) @@ -133,19 +133,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#8)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 6] +(19) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] (20) ReusedExchange [Reuses operator id: 50] Output [1]: [d_date_sk#9] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#13] Right keys [1]: [d_date_sk#9] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#12] Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#9] @@ -153,17 +153,17 @@ Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#9] Input [1]: [cs_ship_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 9] +(24) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#12] Join type: ExistenceJoin(exists#1) Join condition: None -(25) Filter [codegen id : 9] +(25) Filter [codegen id : 4] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(26) Project [codegen id : 9] +(26) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -182,20 +182,20 @@ Condition : isnotnull(ca_address_sk#14) Input [2]: [ca_address_sk#14, ca_state#15] Arguments: [ca_address_sk#14, ca_state#16], [ca_address_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#15, 2, true, false, true) AS ca_state#16] -(30) CometColumnarToRow [codegen id : 7] +(30) CometNativeColumnarToRow Input [2]: [ca_address_sk#14, ca_state#16] (31) BroadcastExchange Input [2]: [ca_address_sk#14, ca_state#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(32) BroadcastHashJoin [codegen id : 9] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#14] Join type: Inner Join condition: None -(33) Project [codegen id : 9] +(33) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#4, ca_state#16] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#14, ca_state#16] @@ -214,24 +214,24 @@ Condition : isnotnull(cd_demo_sk#17) Input [6]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Arguments: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22], [cd_demo_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#18, 1, true, false, true) AS cd_gender#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -(37) CometColumnarToRow [codegen id : 8] +(37) CometNativeColumnarToRow Input [6]: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] (38) BroadcastExchange Input [6]: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(39) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#17] Join type: Inner Join condition: None -(40) Project [codegen id : 9] +(40) Project [codegen id : 4] Output [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Input [8]: [c_current_cdemo_sk#4, ca_state#16, cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -(41) HashAggregate [codegen id : 9] +(41) HashAggregate [codegen id : 4] Input [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Keys [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#20), partial_max(cd_dep_count#20), partial_sum(cd_dep_count#20), partial_avg(cd_dep_employed_count#21), partial_max(cd_dep_employed_count#21), partial_sum(cd_dep_employed_count#21), partial_avg(cd_dep_college_count#22), partial_max(cd_dep_college_count#22), partial_sum(cd_dep_college_count#22)] @@ -242,10 +242,10 @@ Results [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50] Arguments: hashpartitioning(ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(43) CometColumnarToRow [codegen id : 10] +(43) CometNativeColumnarToRow Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50] -(44) HashAggregate [codegen id : 10] +(44) HashAggregate [codegen id : 5] Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50] Keys [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] Functions [10]: [count(1), avg(cd_dep_count#20), max(cd_dep_count#20), sum(cd_dep_count#20), avg(cd_dep_employed_count#21), max(cd_dep_employed_count#21), sum(cd_dep_employed_count#21), avg(cd_dep_college_count#22), max(cd_dep_college_count#22), sum(cd_dep_college_count#22)] @@ -260,7 +260,7 @@ Arguments: 100, [ca_state#16 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * CometColumnarToRow (49) ++- CometNativeColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometNativeScan parquet spark_catalog.default.date_dim (46) @@ -281,7 +281,7 @@ Condition : ((((isnotnull(d_year#73) AND isnotnull(d_qoy#74)) AND (d_year#73 = 2 Input [3]: [d_date_sk#9, d_year#73, d_qoy#74] Arguments: [d_date_sk#9], [d_date_sk#9] -(49) CometColumnarToRow [codegen id : 1] +(49) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (50) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt index 8935b6cc13..40a8ef4a6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- BroadcastHashJoin : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- BroadcastExchange @@ -22,12 +22,12 @@ TakeOrderedAndProject : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,7 +38,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,17 +49,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/simplified.txt index 0fb95102c3..e7163949cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -15,13 +15,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -29,17 +29,15 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -50,7 +48,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -61,17 +59,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/explain.txt index 5df7e6870f..b270ee1a61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#15] Input [1]: [cs_ship_customer_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#15] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#18) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#18, ca_state#20] (33) BroadcastExchange Input [2]: [ca_address_sk#18, ca_state#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#20] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#21) Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] (40) BroadcastExchange Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#21] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/simplified.txt index fea6a32741..af19f84961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 5df7e6870f..b270ee1a61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -37,12 +37,12 @@ TakeOrderedAndProject (47) : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) : : +- ReusedExchange (21) : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) + : +- CometNativeColumnarToRow (32) : +- CometProject (31) : +- CometFilter (30) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) +- BroadcastExchange (40) - +- * CometColumnarToRow (39) + +- CometNativeColumnarToRow (39) +- CometProject (38) +- CometFilter (37) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) @@ -103,7 +103,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) CometColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,7 +132,7 @@ Input [1]: [ws_bill_customer_sk#12] Input [1]: [ws_bill_customer_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(19) BroadcastHashJoin [codegen id : 5] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [ws_bill_customer_sk#12] Join type: ExistenceJoin(exists#2) @@ -164,17 +164,17 @@ Input [1]: [cs_ship_customer_sk#15] Input [1]: [cs_ship_customer_sk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(26) BroadcastHashJoin [codegen id : 5] +(26) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_customer_sk#3] Right keys [1]: [cs_ship_customer_sk#15] Join type: ExistenceJoin(exists#1) Join condition: None -(27) Filter [codegen id : 5] +(27) Filter [codegen id : 3] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] Condition : (exists#2 OR exists#1) -(28) Project [codegen id : 5] +(28) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] @@ -193,20 +193,20 @@ Condition : isnotnull(ca_address_sk#18) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] -(32) CometColumnarToRow [codegen id : 3] +(32) CometNativeColumnarToRow Input [2]: [ca_address_sk#18, ca_state#20] (33) BroadcastExchange Input [2]: [ca_address_sk#18, ca_state#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(34) BroadcastHashJoin [codegen id : 5] +(34) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_addr_sk#5] Right keys [1]: [ca_address_sk#18] Join type: Inner Join condition: None -(35) Project [codegen id : 5] +(35) Project [codegen id : 3] Output [2]: [c_current_cdemo_sk#4, ca_state#20] Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] @@ -225,24 +225,24 @@ Condition : isnotnull(cd_demo_sk#21) Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(39) CometColumnarToRow [codegen id : 4] +(39) CometNativeColumnarToRow Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] (40) BroadcastExchange Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(41) BroadcastHashJoin [codegen id : 5] +(41) BroadcastHashJoin [codegen id : 3] Left keys [1]: [c_current_cdemo_sk#4] Right keys [1]: [cd_demo_sk#21] Join type: Inner Join condition: None -(42) Project [codegen id : 5] +(42) Project [codegen id : 3] Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -(43) HashAggregate [codegen id : 5] +(43) HashAggregate [codegen id : 3] Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] @@ -253,10 +253,10 @@ Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(45) CometColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [codegen id : 4] Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -(46) HashAggregate [codegen id : 6] +(46) HashAggregate [codegen id : 4] Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] @@ -271,7 +271,7 @@ Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) @@ -292,7 +292,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2 Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] Arguments: [d_date_sk#9], [d_date_sk#9] -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [1]: [d_date_sk#9] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt index a6f33d6f7e..855ca74e0b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,12 +50,12 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt index fea6a32741..af19f84961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometColumnarToRow InputAdapter CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) + WholeStageCodegen (3) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -25,12 +25,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] @@ -57,17 +55,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/explain.txt index fbea598466..8064d27539 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (43) +- * HashAggregate (42) - +- * CometColumnarToRow (41) + +- CometNativeColumnarToRow (41) +- CometColumnarExchange (40) +- * HashAggregate (39) +- * Project (38) @@ -11,7 +11,7 @@ TakeOrderedAndProject (43) : :- * Project (24) : : +- * BroadcastHashJoin LeftSemi BuildRight (23) : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) + : : : :- CometNativeColumnarToRow (3) : : : : +- CometFilter (2) : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) : : : +- BroadcastExchange (9) @@ -33,12 +33,12 @@ TakeOrderedAndProject (43) : : : +- Scan parquet spark_catalog.default.catalog_sales (16) : : +- ReusedExchange (18) : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) + : +- CometNativeColumnarToRow (28) : +- CometProject (27) : +- CometFilter (26) : +- CometNativeScan parquet spark_catalog.default.customer_address (25) +- BroadcastExchange (36) - +- * CometColumnarToRow (35) + +- CometNativeColumnarToRow (35) +- CometProject (34) +- CometFilter (33) +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) @@ -55,7 +55,7 @@ ReadSchema: struct -(5) ColumnarToRow [codegen id : 2] +(5) ColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] (6) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] -(7) BroadcastHashJoin [codegen id : 2] +(7) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(8) Project [codegen id : 2] +(8) Project [codegen id : 1] Output [1]: [ss_customer_sk#4] Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] @@ -85,7 +85,7 @@ Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] Input [1]: [ss_customer_sk#4] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(10) BroadcastHashJoin [codegen id : 9] +(10) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#4] Join type: LeftSemi @@ -98,19 +98,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 4] +(12) ColumnarToRow [codegen id : 2] Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] (13) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#7] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#9] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [1]: [ws_bill_customer_sk#8 AS customsk#10] Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#7] @@ -121,19 +121,19 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#6)] ReadSchema: struct -(17) ColumnarToRow [codegen id : 6] +(17) ColumnarToRow [codegen id : 3] Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] (18) ReusedExchange [Reuses operator id: 48] Output [1]: [d_date_sk#13] -(19) BroadcastHashJoin [codegen id : 6] +(19) BroadcastHashJoin [codegen id : 3] Left keys [1]: [cs_sold_date_sk#12] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 6] +(20) Project [codegen id : 3] Output [1]: [cs_ship_customer_sk#11 AS customsk#14] Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] @@ -143,13 +143,13 @@ Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] Input [1]: [customsk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(23) BroadcastHashJoin [codegen id : 9] +(23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#1] Right keys [1]: [customsk#10] Join type: LeftSemi Join condition: None -(24) Project [codegen id : 9] +(24) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] @@ -168,20 +168,20 @@ Condition : isnotnull(ca_address_sk#15) Input [2]: [ca_address_sk#15, ca_state#16] Arguments: [ca_address_sk#15, ca_state#17], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#17] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [2]: [ca_address_sk#15, ca_state#17] (29) BroadcastExchange Input [2]: [ca_address_sk#15, ca_state#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(30) BroadcastHashJoin [codegen id : 9] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_addr_sk#3] Right keys [1]: [ca_address_sk#15] Join type: Inner Join condition: None -(31) Project [codegen id : 9] +(31) Project [codegen id : 4] Output [2]: [c_current_cdemo_sk#2, ca_state#17] Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#15, ca_state#17] @@ -200,24 +200,24 @@ Condition : isnotnull(cd_demo_sk#18) Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(35) CometColumnarToRow [codegen id : 8] +(35) CometNativeColumnarToRow Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] (36) BroadcastExchange Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(37) BroadcastHashJoin [codegen id : 9] +(37) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_current_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(38) Project [codegen id : 4] Output [6]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Input [8]: [c_current_cdemo_sk#2, ca_state#17, cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -(39) HashAggregate [codegen id : 9] +(39) HashAggregate [codegen id : 4] Input [6]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Keys [6]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [partial_count(1), partial_avg(cd_dep_count#21), partial_max(cd_dep_count#21), partial_sum(cd_dep_count#21), partial_avg(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_sum(cd_dep_employed_count#22), partial_avg(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_sum(cd_dep_college_count#23)] @@ -228,10 +228,10 @@ Results [19]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, Input [19]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Arguments: hashpartitioning(ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [19]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -(42) HashAggregate [codegen id : 10] +(42) HashAggregate [codegen id : 5] Input [19]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] Keys [6]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] Functions [10]: [count(1), avg(cd_dep_count#21), max(cd_dep_count#21), sum(cd_dep_count#21), avg(cd_dep_employed_count#22), max(cd_dep_employed_count#22), sum(cd_dep_employed_count#22), avg(cd_dep_college_count#23), max(cd_dep_college_count#23), sum(cd_dep_college_count#23)] @@ -246,7 +246,7 @@ Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_m Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometNativeScan parquet spark_catalog.default.date_dim (44) @@ -267,7 +267,7 @@ Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 1 Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt index 8aebe6be9a..9091536e96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : :- Project : : +- BroadcastHashJoin : : :- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- BroadcastExchange @@ -20,12 +20,12 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,7 +37,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -47,17 +47,17 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/simplified.txt index bed891a0a3..4fe5cb9c56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/simplified.txt @@ -1,10 +1,10 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) + WholeStageCodegen (5) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) + WholeStageCodegen (4) HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] @@ -13,13 +13,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Project [c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_customer_sk,customsk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] InputAdapter BroadcastExchange #2 - WholeStageCodegen (2) + WholeStageCodegen (1) Project [ss_customer_sk] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] ColumnarToRow @@ -27,18 +27,16 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #4 Union - WholeStageCodegen (4) + WholeStageCodegen (2) Project [ws_bill_customer_sk] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] ColumnarToRow @@ -47,7 +45,7 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) + WholeStageCodegen (3) Project [cs_ship_customer_sk] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] ColumnarToRow @@ -58,17 +56,13 @@ TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_ ReusedExchange [d_date_sk] #3 InputAdapter BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometNativeColumnarToRow + CometProject [ca_state] [ca_address_sk,ca_state] + CometFilter [ca_address_sk,ca_state] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] InputAdapter BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeColumnarToRow + CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/explain.txt index 97c7d7659b..5a36668686 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/explain.txt @@ -232,7 +232,7 @@ Input [18]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, c Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999 Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/simplified.txt index 69e8d4868c..15668eeef0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/simplified.txt @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 97c7d7659b..5a36668686 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -232,7 +232,7 @@ Input [18]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, c Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -253,7 +253,7 @@ Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999 Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (45) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/extended.txt index 2cdc75e15e..fc330dd8b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 69e8d4868c..15668eeef0 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 @@ -20,12 +20,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] CometBroadcastExchange [d_date_sk] #4 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/explain.txt index 0f517df9a4..5b90ab9951 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- CometExchange (43) +- CometHashAggregate (42) @@ -10,7 +10,7 @@ TakeOrderedAndProject (48) +- * HashAggregate (40) +- Union (39) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -24,28 +24,28 @@ TakeOrderedAndProject (48) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.item (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.store (14) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) + : +- CometNativeColumnarToRow (26) : +- ReusedExchange (25) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * HashAggregate (34) - +- * CometColumnarToRow (33) + +- CometNativeColumnarToRow (33) +- ReusedExchange (32) @@ -57,23 +57,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) (4) ReusedExchange [Reuses operator id: 53] Output [1]: [d_date_sk#7] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] @@ -92,20 +92,20 @@ Condition : isnotnull(i_item_sk#8) Input [3]: [i_item_sk#8, i_class#9, i_category#10] Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#8, i_class#11, i_category#12] (11) BroadcastExchange Input [3]: [i_item_sk#8, i_class#11, i_category#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#8] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] @@ -124,24 +124,24 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [s_store_sk#13, s_state#14] Arguments: [s_store_sk#13], [s_store_sk#13] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [1]: [s_store_sk#13] (18) BroadcastExchange Input [1]: [s_store_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#13] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] Keys [2]: [i_category#12, i_class#11] Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] @@ -152,10 +152,10 @@ Results [4]: [i_category#12, i_class#11, sum#17, sum#18] Input [4]: [i_category#12, i_class#11, sum#17, sum#18] Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 5] +(23) CometNativeColumnarToRow Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [4]: [i_category#12, i_class#11, sum#17, sum#18] Keys [2]: [i_category#12, i_class#11] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] @@ -165,17 +165,17 @@ Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / M (25) ReusedExchange [Reuses operator id: 22] Output [4]: [i_category#12, i_class#11, sum#27, sum#28] -(26) CometColumnarToRow [codegen id : 10] +(26) CometNativeColumnarToRow Input [4]: [i_category#12, i_class#11, sum#27, sum#28] -(27) HashAggregate [codegen id : 10] +(27) HashAggregate [codegen id : 4] Input [4]: [i_category#12, i_class#11, sum#27, sum#28] Keys [2]: [i_category#12, i_class#11] Functions [2]: [sum(UnscaledValue(ss_net_profit#29)), sum(UnscaledValue(ss_ext_sales_price#30))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#29))#31, sum(UnscaledValue(ss_ext_sales_price#30))#32] Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#29))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#30))#32,17,2) AS ss_ext_sales_price#34, i_category#12] -(28) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 4] Input [3]: [ss_net_profit#33, ss_ext_sales_price#34, i_category#12] Keys [1]: [i_category#12] Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] @@ -186,10 +186,10 @@ Results [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 11] +(30) CometNativeColumnarToRow Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -(31) HashAggregate [codegen id : 11] +(31) HashAggregate [codegen id : 5] Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] Keys [1]: [i_category#12] Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] @@ -199,17 +199,17 @@ Results [6]: [cast((sum(ss_net_profit#33)#43 / sum(ss_ext_sales_price#34)#44) as (32) ReusedExchange [Reuses operator id: 22] Output [4]: [i_category#12, i_class#11, sum#50, sum#51] -(33) CometColumnarToRow [codegen id : 16] +(33) CometNativeColumnarToRow Input [4]: [i_category#12, i_class#11, sum#50, sum#51] -(34) HashAggregate [codegen id : 16] +(34) HashAggregate [codegen id : 7] Input [4]: [i_category#12, i_class#11, sum#50, sum#51] Keys [2]: [i_category#12, i_class#11] Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#31, sum(UnscaledValue(ss_ext_sales_price#53))#32] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#32,17,2) AS ss_ext_sales_price#34] -(35) HashAggregate [codegen id : 16] +(35) HashAggregate [codegen id : 7] Input [2]: [ss_net_profit#33, ss_ext_sales_price#34] Keys: [] Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] @@ -220,10 +220,10 @@ Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 17] +(37) CometNativeColumnarToRow Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] -(38) HashAggregate [codegen id : 17] +(38) HashAggregate [codegen id : 8] Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] Keys: [] Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] @@ -232,7 +232,7 @@ Results [6]: [cast((sum(ss_net_profit#33)#62 / sum(ss_ext_sales_price#34)#63) as (39) Union -(40) HashAggregate [codegen id : 18] +(40) HashAggregate [codegen id : 9] Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] Functions: [] @@ -256,14 +256,14 @@ Arguments: hashpartitioning(lochierarchy#26, _w0#70, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70] Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70], [lochierarchy#26 ASC NULLS FIRST, _w0#70 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 19] +(45) CometNativeColumnarToRow Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70] (46) Window Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70] Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#70, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#71], [lochierarchy#26, _w0#70], [gross_margin#21 ASC NULLS FIRST] -(47) Project [codegen id : 20] +(47) Project [codegen id : 10] Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#71] Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70, rank_within_parent#71] @@ -275,7 +275,7 @@ Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (53) -+- * CometColumnarToRow (52) ++- CometNativeColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometNativeScan parquet spark_catalog.default.date_dim (49) @@ -296,7 +296,7 @@ Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#7, d_year#72] Arguments: [d_date_sk#7], [d_date_sk#7] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (53) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt index 7d9e9883d2..1fc86dc5a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -9,7 +9,7 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -23,31 +23,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -61,31 +61,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -99,22 +99,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/simplified.txt index 65728fc6f2..a101d9fb6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/simplified.txt @@ -1,81 +1,73 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (20) + WholeStageCodegen (10) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - 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 (18) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + 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 (9) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + ReusedExchange [d_date_sk] #4 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (5) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category] #7 + WholeStageCodegen (4) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (8) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #8 + WholeStageCodegen (7) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt index 0f966424f8..079fb1c2e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt @@ -244,7 +244,7 @@ Arguments: 100, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -265,7 +265,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt index 1f6c984b22..968ea0611f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,7 +53,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -85,7 +85,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt index e91b278c14..9fddc4902a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt @@ -25,12 +25,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt index 0f966424f8..079fb1c2e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt @@ -244,7 +244,7 @@ Arguments: 100, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (48) -+- * CometColumnarToRow (47) ++- CometNativeColumnarToRow (47) +- CometProject (46) +- CometFilter (45) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) @@ -265,7 +265,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(47) CometColumnarToRow [codegen id : 1] +(47) CometNativeColumnarToRow Input [1]: [d_date_sk#7] (48) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt index 1f6c984b22..968ea0611f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,7 +53,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -85,7 +85,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 e91b278c14..9fddc4902a 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 @@ -25,12 +25,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/explain.txt index 9613d03c41..67b73fe5c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.store_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.store (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.store_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [ss_item_sk#6] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] (17) BroadcastExchange Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_store_sk#7] Right keys [1]: [s_store_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] @@ -151,10 +151,10 @@ Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_yea Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] (27) Window Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] (32) ReusedExchange [Reuses operator id: 21] Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_ Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] (38) Window Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] @@ -228,13 +228,13 @@ Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year# Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] @@ -245,14 +245,14 @@ Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] (46) Window Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] @@ -260,13 +260,13 @@ Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year# Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt index 092083e465..ab44bbef5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/simplified.txt index ee8e4dd40c..404ef436b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk,s_store_name,s_company_name] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt index 7b1f860f32..bb0182c8d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt index a62c33ecc1..8a10571eff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 7b1f860f32..bb0182c8d4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt index b50b570b4b..f38ceaf317 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 a62c33ecc1..8a10571eff 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/explain.txt index 070113d8ea..9f67fa1915 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (84) +CometNativeColumnarToRow (84) +- CometTakeOrderedAndProject (83) +- CometHashAggregate (82) +- CometColumnarExchange (81) @@ -10,11 +10,11 @@ : +- Window (24) : +- * Sort (23) : +- Window (22) - : +- * CometColumnarToRow (21) + : +- CometNativeColumnarToRow (21) : +- CometSort (20) : +- CometColumnarExchange (19) : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometColumnarExchange (16) : +- * HashAggregate (15) : +- * Project (14) @@ -26,7 +26,7 @@ : : : +- * Filter (3) : : : +- * ColumnarToRow (2) : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) + : : +- CometNativeColumnarToRow (9) : : +- CometProject (8) : : +- CometFilter (7) : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) @@ -36,11 +36,11 @@ : +- Window (50) : +- * Sort (49) : +- Window (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometSort (46) : +- CometColumnarExchange (45) : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) + : +- CometNativeColumnarToRow (43) : +- CometColumnarExchange (42) : +- * HashAggregate (41) : +- * Project (40) @@ -52,7 +52,7 @@ : : : +- * Filter (29) : : : +- * ColumnarToRow (28) : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) + : : +- CometNativeColumnarToRow (35) : : +- CometProject (34) : : +- CometFilter (33) : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) @@ -62,11 +62,11 @@ +- Window (76) +- * Sort (75) +- Window (74) - +- * CometColumnarToRow (73) + +- CometNativeColumnarToRow (73) +- CometSort (72) +- CometColumnarExchange (71) +- * HashAggregate (70) - +- * CometColumnarToRow (69) + +- CometNativeColumnarToRow (69) +- CometColumnarExchange (68) +- * HashAggregate (67) +- * Project (66) @@ -78,7 +78,7 @@ : : +- * Filter (55) : : +- * ColumnarToRow (54) : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) + : +- CometNativeColumnarToRow (61) : +- CometProject (60) : +- CometFilter (59) : +- CometNativeScan parquet spark_catalog.default.store_returns (58) @@ -123,33 +123,33 @@ Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AN Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(9) CometColumnarToRow +(9) CometNativeColumnarToRow Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -(10) BroadcastHashJoin [codegen id : 3] +(10) BroadcastHashJoin [codegen id : 2] Left keys [2]: [ws_order_number#2, ws_item_sk#1] Right keys [2]: [wr_order_number#9, wr_item_sk#8] Join type: Inner Join condition: None -(11) Project [codegen id : 3] +(11) Project [codegen id : 2] Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] (12) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#13] -(13) BroadcastHashJoin [codegen id : 3] +(13) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ws_sold_date_sk#6] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(14) Project [codegen id : 3] +(14) Project [codegen id : 2] Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -(15) HashAggregate [codegen id : 3] +(15) HashAggregate [codegen id : 2] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] Keys [1]: [ws_item_sk#1] Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -160,10 +160,10 @@ Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty# Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(17) CometColumnarToRow [codegen id : 4] +(17) CometNativeColumnarToRow Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -(18) HashAggregate [codegen id : 4] +(18) HashAggregate [codegen id : 3] Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] Keys [1]: [ws_item_sk#1] Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] @@ -178,14 +178,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 5] +(21) CometNativeColumnarToRow Input [3]: [item#30, return_ratio#31, currency_ratio#32] (22) Window Input [3]: [item#30, return_ratio#31, currency_ratio#32] Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] -(23) Sort [codegen id : 6] +(23) Sort [codegen id : 4] Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 @@ -193,11 +193,11 @@ Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] -(25) Filter [codegen id : 7] +(25) Filter [codegen id : 5] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) -(26) Project [codegen id : 7] +(26) Project [codegen id : 5] Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] @@ -209,14 +209,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -(29) Filter [codegen id : 8] +(29) Filter [codegen id : 6] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) -(30) Project [codegen id : 8] +(30) Project [codegen id : 6] Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] @@ -239,33 +239,33 @@ Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.0 Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(35) CometColumnarToRow +(35) CometNativeColumnarToRow Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 7] Left keys [2]: [cs_order_number#37, cs_item_sk#36] Right keys [2]: [cr_order_number#43, cr_item_sk#42] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 7] Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] (38) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#47] -(39) BroadcastHashJoin [codegen id : 10] +(39) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cs_sold_date_sk#41] Right keys [1]: [d_date_sk#47] Join type: Inner Join condition: None -(40) Project [codegen id : 10] +(40) Project [codegen id : 7] Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] -(41) HashAggregate [codegen id : 10] +(41) HashAggregate [codegen id : 7] Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] Keys [1]: [cs_item_sk#36] Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -276,10 +276,10 @@ Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) CometColumnarToRow [codegen id : 11] +(43) CometNativeColumnarToRow Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -(44) HashAggregate [codegen id : 11] +(44) HashAggregate [codegen id : 8] Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] Keys [1]: [cs_item_sk#36] Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] @@ -294,14 +294,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 12] +(47) CometNativeColumnarToRow Input [3]: [item#64, return_ratio#65, currency_ratio#66] (48) Window Input [3]: [item#64, return_ratio#65, currency_ratio#66] Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] -(49) Sort [codegen id : 13] +(49) Sort [codegen id : 9] Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 @@ -309,11 +309,11 @@ Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] -(51) Filter [codegen id : 14] +(51) Filter [codegen id : 10] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) -(52) Project [codegen id : 14] +(52) Project [codegen id : 10] Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] @@ -325,14 +325,14 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(54) ColumnarToRow [codegen id : 15] +(54) ColumnarToRow [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -(55) Filter [codegen id : 15] +(55) Filter [codegen id : 11] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) -(56) Project [codegen id : 15] +(56) Project [codegen id : 11] Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] @@ -355,33 +355,33 @@ Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AN Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(61) CometColumnarToRow +(61) CometNativeColumnarToRow Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] -(62) BroadcastHashJoin [codegen id : 17] +(62) BroadcastHashJoin [codegen id : 12] Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] Join type: Inner Join condition: None -(63) Project [codegen id : 17] +(63) Project [codegen id : 12] Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] (64) ReusedExchange [Reuses operator id: 89] Output [1]: [d_date_sk#81] -(65) BroadcastHashJoin [codegen id : 17] +(65) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ss_sold_date_sk#75] Right keys [1]: [d_date_sk#81] Join type: Inner Join condition: None -(66) Project [codegen id : 17] +(66) Project [codegen id : 12] Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] -(67) HashAggregate [codegen id : 17] +(67) HashAggregate [codegen id : 12] Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] Keys [1]: [ss_item_sk#70] Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -392,10 +392,10 @@ Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(69) CometColumnarToRow [codegen id : 18] +(69) CometNativeColumnarToRow Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -(70) HashAggregate [codegen id : 18] +(70) HashAggregate [codegen id : 13] Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] Keys [1]: [ss_item_sk#70] Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] @@ -410,14 +410,14 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id= Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [3]: [item#98, return_ratio#99, currency_ratio#100] (74) Window Input [3]: [item#98, return_ratio#99, currency_ratio#100] Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] -(75) Sort [codegen id : 20] +(75) Sort [codegen id : 14] Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 @@ -425,17 +425,17 @@ Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] -(77) Filter [codegen id : 21] +(77) Filter [codegen id : 15] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) -(78) Project [codegen id : 21] +(78) Project [codegen id : 15] Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] (79) Union -(80) HashAggregate [codegen id : 22] +(80) HashAggregate [codegen id : 16] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Functions: [] @@ -455,14 +455,14 @@ Functions: [] Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -(84) CometColumnarToRow [codegen id : 23] +(84) CometNativeColumnarToRow Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometNativeScan parquet spark_catalog.default.date_dim (85) @@ -483,7 +483,7 @@ Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 Input [3]: [d_date_sk#13, d_year#104, d_moy#105] Arguments: [d_date_sk#13], [d_date_sk#13] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index 6c4ded1515..0090739d55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange @@ -9,11 +9,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -27,16 +27,16 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -45,11 +45,11 @@ CometColumnarToRow : +- Window : +- Sort : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -62,12 +62,12 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -76,11 +76,11 @@ CometColumnarToRow +- Window +- Sort +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -93,12 +93,12 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/simplified.txt index 047269434d..018aec1d47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/simplified.txt @@ -1,140 +1,130 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (16) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (5) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (4) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #2 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + CometNativeColumnarToRow + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (10) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (9) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #6 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [cs_item_sk] #7 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (15) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (14) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + CometNativeColumnarToRow + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #9 + WholeStageCodegen (13) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (12) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (11) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt index 74702d596d..b027d714df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt index ca80833ee5..3353e1c331 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 74702d596d..b027d714df 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -427,7 +427,7 @@ Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank# Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometProject (81) +- CometFilter (80) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) @@ -448,7 +448,7 @@ Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2 Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [1]: [d_date_sk#13] (83) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt index 75684a9669..d1b415dfe4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ca80833ee5..3353e1c331 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 @@ -35,12 +35,10 @@ WholeStageCodegen (11) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/explain.txt index 23adfe692b..178350b771 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/explain.txt @@ -6,7 +6,7 @@ TakeOrderedAndProject (77) +- * Project (73) +- * BroadcastHashJoin Inner BuildRight (72) :- Window (65) - : +- * CometColumnarToRow (64) + : +- CometNativeColumnarToRow (64) : +- CometSort (63) : +- CometExchange (62) : +- CometProject (61) @@ -15,18 +15,18 @@ TakeOrderedAndProject (77) : :- CometSort (29) : : +- CometColumnarExchange (28) : : +- * HashAggregate (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometColumnarExchange (25) : : +- * HashAggregate (24) : : +- * Project (23) : : +- * BroadcastHashJoin Inner BuildRight (22) : : :- * Project (15) : : : +- Window (14) - : : : +- * CometColumnarToRow (13) + : : : +- CometNativeColumnarToRow (13) : : : +- CometSort (12) : : : +- CometColumnarExchange (11) : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometColumnarExchange (8) : : : +- * HashAggregate (7) : : : +- * Project (6) @@ -38,24 +38,24 @@ TakeOrderedAndProject (77) : : +- BroadcastExchange (21) : : +- * Project (20) : : +- Window (19) - : : +- * CometColumnarToRow (18) + : : +- CometNativeColumnarToRow (18) : : +- CometSort (17) : : +- ReusedExchange (16) : +- CometSort (58) : +- CometColumnarExchange (57) : +- * HashAggregate (56) - : +- * CometColumnarToRow (55) + : +- CometNativeColumnarToRow (55) : +- CometColumnarExchange (54) : +- * HashAggregate (53) : +- * Project (52) : +- * BroadcastHashJoin Inner BuildRight (51) : :- * Project (44) : : +- Window (43) - : : +- * CometColumnarToRow (42) + : : +- CometNativeColumnarToRow (42) : : +- CometSort (41) : : +- CometColumnarExchange (40) : : +- * HashAggregate (39) - : : +- * CometColumnarToRow (38) + : : +- CometNativeColumnarToRow (38) : : +- CometColumnarExchange (37) : : +- * HashAggregate (36) : : +- * Project (35) @@ -67,13 +67,13 @@ TakeOrderedAndProject (77) : +- BroadcastExchange (50) : +- * Project (49) : +- Window (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- CometSort (46) : +- ReusedExchange (45) +- BroadcastExchange (71) +- * Project (70) +- Window (69) - +- * CometColumnarToRow (68) + +- CometNativeColumnarToRow (68) +- CometSort (67) +- ReusedExchange (66) @@ -86,27 +86,27 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 82] Output [2]: [d_date_sk#5, d_date#6] -(5) BroadcastHashJoin [codegen id : 2] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 2] +(6) Project [codegen id : 1] Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -(7) HashAggregate [codegen id : 2] +(7) HashAggregate [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] @@ -117,10 +117,10 @@ Results [3]: [ws_item_sk#1, d_date#6, sum#8] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(9) CometColumnarToRow [codegen id : 3] +(9) CometNativeColumnarToRow Input [3]: [ws_item_sk#1, d_date#6, sum#8] -(10) HashAggregate [codegen id : 3] +(10) HashAggregate [codegen id : 2] Input [3]: [ws_item_sk#1, d_date#6, sum#8] Keys [2]: [ws_item_sk#1, d_date#6] Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] @@ -135,14 +135,14 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(13) CometColumnarToRow [codegen id : 4] +(13) CometNativeColumnarToRow Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] (14) Window Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Project [codegen id : 10] +(15) Project [codegen id : 6] Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] @@ -153,14 +153,14 @@ Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] -(18) CometColumnarToRow [codegen id : 8] +(18) CometNativeColumnarToRow Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] (19) Window Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] -(20) Project [codegen id : 9] +(20) Project [codegen id : 5] Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] @@ -168,17 +168,17 @@ Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] Input [3]: [item_sk#16, sumws#17, rk#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) BroadcastHashJoin [codegen id : 10] +(22) BroadcastHashJoin [codegen id : 6] Left keys [1]: [item_sk#10] Right keys [1]: [item_sk#16] Join type: Inner Join condition: (rk#12 >= rk#15) -(23) Project [codegen id : 10] +(23) Project [codegen id : 6] Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] -(24) HashAggregate [codegen id : 10] +(24) HashAggregate [codegen id : 6] Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [partial_sum(sumws#17)] @@ -189,10 +189,10 @@ Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometColumnarToRow [codegen id : 11] +(26) CometNativeColumnarToRow Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(27) HashAggregate [codegen id : 11] +(27) HashAggregate [codegen id : 7] Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [sum(sumws#17)] @@ -215,27 +215,27 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 13] +(31) ColumnarToRow [codegen id : 8] Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -(32) Filter [codegen id : 13] +(32) Filter [codegen id : 8] Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#24) (33) ReusedExchange [Reuses operator id: 82] Output [2]: [d_date_sk#27, d_date#28] -(34) BroadcastHashJoin [codegen id : 13] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(35) Project [codegen id : 13] +(35) Project [codegen id : 8] Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27, d_date#28] -(36) HashAggregate [codegen id : 13] +(36) HashAggregate [codegen id : 8] Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] Keys [2]: [ss_item_sk#24, d_date#28] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] @@ -246,10 +246,10 @@ Results [3]: [ss_item_sk#24, d_date#28, sum#30] Input [3]: [ss_item_sk#24, d_date#28, sum#30] Arguments: hashpartitioning(ss_item_sk#24, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(38) CometColumnarToRow [codegen id : 14] +(38) CometNativeColumnarToRow Input [3]: [ss_item_sk#24, d_date#28, sum#30] -(39) HashAggregate [codegen id : 14] +(39) HashAggregate [codegen id : 9] Input [3]: [ss_item_sk#24, d_date#28, sum#30] Keys [2]: [ss_item_sk#24, d_date#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] @@ -264,14 +264,14 @@ Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumna Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] Arguments: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(42) CometColumnarToRow [codegen id : 15] +(42) CometNativeColumnarToRow Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] (43) Window Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#28 ASC NULLS FIRST] -(44) Project [codegen id : 21] +(44) Project [codegen id : 13] Output [4]: [item_sk#32, d_date#28, sumss#33, rk#34] Input [5]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24, rk#34] @@ -282,14 +282,14 @@ Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] Arguments: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36], [ss_item_sk#36 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST] -(47) CometColumnarToRow [codegen id : 19] +(47) CometNativeColumnarToRow Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] (48) Window Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#35 ASC NULLS FIRST] -(49) Project [codegen id : 20] +(49) Project [codegen id : 12] Output [3]: [item_sk#32 AS item_sk#38, sumss#33 AS sumss#39, rk#37] Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] @@ -297,17 +297,17 @@ Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] Input [3]: [item_sk#38, sumss#39, rk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(51) BroadcastHashJoin [codegen id : 21] +(51) BroadcastHashJoin [codegen id : 13] Left keys [1]: [item_sk#32] Right keys [1]: [item_sk#38] Join type: Inner Join condition: (rk#34 >= rk#37) -(52) Project [codegen id : 21] +(52) Project [codegen id : 13] Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#38, sumss#39, rk#37] -(53) HashAggregate [codegen id : 21] +(53) HashAggregate [codegen id : 13] Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] Keys [3]: [item_sk#32, d_date#28, sumss#33] Functions [1]: [partial_sum(sumss#39)] @@ -318,10 +318,10 @@ Results [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Arguments: hashpartitioning(item_sk#32, d_date#28, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(55) CometColumnarToRow [codegen id : 22] +(55) CometNativeColumnarToRow Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] -(56) HashAggregate [codegen id : 22] +(56) HashAggregate [codegen id : 14] Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] Keys [3]: [item_sk#32, d_date#28, sumss#33] Functions [1]: [sum(sumss#39)] @@ -357,7 +357,7 @@ Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] -(64) CometColumnarToRow [codegen id : 23] +(64) CometNativeColumnarToRow Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] (65) Window @@ -371,14 +371,14 @@ Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] -(68) CometColumnarToRow [codegen id : 46] +(68) CometNativeColumnarToRow Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] (69) Window Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#46], [d_date#47 ASC NULLS FIRST] -(70) Project [codegen id : 47] +(70) Project [codegen id : 29] Output [4]: [item_sk#46 AS item_sk#52, web_sales#48 AS web_sales#53, store_sales#49 AS store_sales#54, rk#51] Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] @@ -386,31 +386,31 @@ Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] Input [4]: [item_sk#52, web_sales#53, store_sales#54, rk#51] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(72) BroadcastHashJoin [codegen id : 48] +(72) BroadcastHashJoin [codegen id : 30] Left keys [1]: [item_sk#46] Right keys [1]: [item_sk#52] Join type: Inner Join condition: (rk#50 >= rk#51) -(73) Project [codegen id : 48] +(73) Project [codegen id : 30] Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#52, web_sales#53, store_sales#54, rk#51] -(74) HashAggregate [codegen id : 48] +(74) HashAggregate [codegen id : 30] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] Aggregate Attributes [2]: [max#55, max#56] Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] -(75) HashAggregate [codegen id : 48] +(75) HashAggregate [codegen id : 30] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] Functions [2]: [max(web_sales#53), max(store_sales#54)] Aggregate Attributes [2]: [max(web_sales#53)#59, max(store_sales#54)#60] Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#59 AS web_cumulative#61, max(store_sales#54)#60 AS store_cumulative#62] -(76) Filter [codegen id : 48] +(76) Filter [codegen id : 30] Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] Condition : ((isnotnull(web_cumulative#61) AND isnotnull(store_cumulative#62)) AND (web_cumulative#61 > store_cumulative#62)) @@ -422,7 +422,7 @@ Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (82) -+- * CometColumnarToRow (81) ++- CometNativeColumnarToRow (81) +- CometProject (80) +- CometFilter (79) +- CometNativeScan parquet spark_catalog.default.date_dim (78) @@ -443,7 +443,7 @@ Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_mo Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(81) CometColumnarToRow [codegen id : 1] +(81) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (82) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index 8eb3f067e2..92ee33eda6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt @@ -5,7 +5,7 @@ TakeOrderedAndProject +- Project +- BroadcastHashJoin :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometExchange : +- CometProject @@ -14,18 +14,18 @@ TakeOrderedAndProject : :- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometSort : : : +- CometColumnarExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -35,23 +35,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -61,30 +61,30 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -94,18 +94,18 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -115,14 +115,14 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -131,18 +131,18 @@ TakeOrderedAndProject :- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -152,23 +152,23 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -178,30 +178,30 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -211,18 +211,18 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -232,7 +232,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/simplified.txt index 00cd0a509c..55c9b21954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (48) + WholeStageCodegen (30) Filter [web_cumulative,store_cumulative] HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] @@ -7,121 +7,107 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #8 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (21) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #11 - WholeStageCodegen (13) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (20) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + CometNativeColumnarToRow + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (7) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (6) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ws_item_sk,d_date] #5 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #8 + WholeStageCodegen (14) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [item_sk,d_date,sumss] #9 + WholeStageCodegen (13) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #10 + WholeStageCodegen (9) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [ss_item_sk,d_date] #11 + WholeStageCodegen (8) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (12) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + CometNativeColumnarToRow + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 InputAdapter BroadcastExchange #13 - WholeStageCodegen (47) + WholeStageCodegen (29) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (46) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometNativeColumnarToRow + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt index f8a95a6db8..a894d9e3e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometProject (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -436,7 +436,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt index f7d3371108..05c8752bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt @@ -33,7 +33,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -138,7 +138,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -161,7 +161,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt index b3013059b0..bcb2317588 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt @@ -43,12 +43,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index f8a95a6db8..a894d9e3e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -415,7 +415,7 @@ Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_s Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (81) -+- * CometColumnarToRow (80) ++- CometNativeColumnarToRow (80) +- CometProject (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -436,7 +436,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(80) CometNativeColumnarToRow Input [2]: [d_date_sk#5, d_date#6] (81) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt index f7d3371108..05c8752bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt @@ -33,7 +33,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,7 +56,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -138,7 +138,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -161,7 +161,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 b3013059b0..bcb2317588 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 @@ -43,12 +43,10 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] CometBroadcastExchange [d_date_sk,d_date] #7 CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/explain.txt index be90722d2e..e3e412b4ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/explain.txt @@ -9,11 +9,11 @@ TakeOrderedAndProject (51) : : +- Window (29) : : +- * Filter (28) : : +- Window (27) - : : +- * CometColumnarToRow (26) + : : +- CometNativeColumnarToRow (26) : : +- CometSort (25) : : +- CometColumnarExchange (24) : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -22,7 +22,7 @@ TakeOrderedAndProject (51) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) @@ -32,22 +32,22 @@ TakeOrderedAndProject (51) : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) : : : +- ReusedExchange (11) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometFilter (15) : : +- CometNativeScan parquet spark_catalog.default.call_center (14) : +- BroadcastExchange (40) : +- * Project (39) : +- Window (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometSort (36) : +- CometColumnarExchange (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) +- BroadcastExchange (48) +- * Project (47) +- Window (46) - +- * CometColumnarToRow (45) + +- CometNativeColumnarToRow (45) +- CometSort (44) +- ReusedExchange (43) @@ -67,7 +67,7 @@ Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache Input [3]: [i_item_sk#1, i_brand#2, i_category#3] Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] -(4) CometColumnarToRow [codegen id : 4] +(4) CometNativeColumnarToRow Input [3]: [i_item_sk#1, i_brand#4, i_category#5] (5) Scan parquet spark_catalog.default.catalog_sales @@ -89,26 +89,26 @@ Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 4] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [i_item_sk#1] Right keys [1]: [cs_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 4] +(10) Project [codegen id : 2] Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] (11) ReusedExchange [Reuses operator id: 55] Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_sold_date_sk#9] Right keys [1]: [d_date_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] @@ -123,24 +123,24 @@ ReadSchema: struct Input [2]: [cc_call_center_sk#14, cc_name#15] Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) -(16) CometColumnarToRow [codegen id : 3] +(16) CometNativeColumnarToRow Input [2]: [cc_call_center_sk#14, cc_name#15] (17) BroadcastExchange Input [2]: [cc_call_center_sk#14, cc_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 2] Left keys [1]: [cs_call_center_sk#6] Right keys [1]: [cc_call_center_sk#14] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 2] Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -(20) HashAggregate [codegen id : 4] +(20) HashAggregate [codegen id : 2] Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] @@ -151,10 +151,10 @@ Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(22) CometColumnarToRow [codegen id : 5] +(22) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -(23) HashAggregate [codegen id : 5] +(23) HashAggregate [codegen id : 3] Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] @@ -169,14 +169,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 6] +(26) CometNativeColumnarToRow Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] (27) Window Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 4] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) @@ -184,21 +184,21 @@ Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(30) Filter [codegen id : 22] +(30) Filter [codegen id : 13] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(31) Project [codegen id : 22] +(31) Project [codegen id : 13] Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] (32) ReusedExchange [Reuses operator id: 21] Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(33) CometColumnarToRow [codegen id : 12] +(33) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -(34) HashAggregate [codegen id : 12] +(34) HashAggregate [codegen id : 7] Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] @@ -213,14 +213,14 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQU Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(37) CometColumnarToRow [codegen id : 13] +(37) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] (38) Window Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(39) Project [codegen id : 14] +(39) Project [codegen id : 8] Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] @@ -228,13 +228,13 @@ Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales# Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] -(41) BroadcastHashJoin [codegen id : 22] +(41) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] Join type: Inner Join condition: None -(42) Project [codegen id : 22] +(42) Project [codegen id : 13] Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] @@ -245,14 +245,14 @@ Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 20] +(45) CometNativeColumnarToRow Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] (46) Window Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(47) Project [codegen id : 21] +(47) Project [codegen id : 12] Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] @@ -260,13 +260,13 @@ Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales# Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 22] +(49) BroadcastHashJoin [codegen id : 13] Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] Join type: Inner Join condition: None -(50) Project [codegen id : 22] +(50) Project [codegen id : 13] Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] @@ -278,7 +278,7 @@ Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (55) -+- * CometColumnarToRow (54) ++- CometNativeColumnarToRow (54) +- CometFilter (53) +- CometNativeScan parquet spark_catalog.default.date_dim (52) @@ -294,7 +294,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(54) CometColumnarToRow [codegen id : 1] +(54) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (55) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt index 65c66a7da8..66c510aaa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt @@ -8,11 +8,11 @@ TakeOrderedAndProject : : +- Window : : +- Filter : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -31,25 +31,25 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -68,25 +68,25 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -105,15 +105,15 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/simplified.txt index e603f05322..b61b5372fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (22) + WholeStageCodegen (13) Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] @@ -8,80 +8,70 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) + WholeStageCodegen (4) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (3) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 + CometNativeColumnarToRow + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [cs_item_sk,cs_call_center_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [cc_call_center_sk,cc_name] + CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (14) + WholeStageCodegen (8) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #8 - WholeStageCodegen (21) + WholeStageCodegen (12) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 + CometNativeColumnarToRow + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt index d1daee98c1..fcd3657557 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt index d655789fe6..d89e7759f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index d1daee98c1..fcd3657557 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -263,7 +263,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (52) -+- * CometColumnarToRow (51) ++- CometNativeColumnarToRow (51) +- CometFilter (50) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) @@ -279,7 +279,7 @@ ReadSchema: struct Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(51) CometNativeColumnarToRow Input [3]: [d_date_sk#11, d_year#12, d_moy#13] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt index de7d26cd06..1c37fd4e8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -60,7 +60,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -92,7 +92,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometBroadcastExchange 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 d655789fe6..d89e7759f3 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 @@ -34,11 +34,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometNativeColumnarToRow + 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] CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/explain.txt index 6b57e4b9c6..015a244a10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/explain.txt @@ -1,17 +1,17 @@ == Physical Plan == -* CometColumnarToRow (99) +CometNativeColumnarToRow (99) +- CometTakeOrderedAndProject (98) +- CometHashAggregate (97) +- CometColumnarExchange (96) +- * HashAggregate (95) +- Union (94) :- * HashAggregate (79) - : +- * CometColumnarToRow (78) + : +- CometNativeColumnarToRow (78) : +- CometColumnarExchange (77) : +- * HashAggregate (76) : +- Union (75) : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) + : : +- CometNativeColumnarToRow (22) : : +- CometColumnarExchange (21) : : +- * HashAggregate (20) : : +- * Project (19) @@ -29,12 +29,12 @@ : : : : +- Scan parquet spark_catalog.default.store_returns (5) : : : +- ReusedExchange (10) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometProject (15) : : +- CometFilter (14) : : +- CometNativeScan parquet spark_catalog.default.store (13) : :- * HashAggregate (46) - : : +- * CometColumnarToRow (45) + : : +- CometNativeColumnarToRow (45) : : +- CometColumnarExchange (44) : : +- * HashAggregate (43) : : +- * Project (42) @@ -52,12 +52,12 @@ : : : : +- Scan parquet spark_catalog.default.catalog_returns (28) : : : +- ReusedExchange (33) : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) + : : +- CometNativeColumnarToRow (39) : : +- CometProject (38) : : +- CometFilter (37) : : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) : +- * HashAggregate (74) - : +- * CometColumnarToRow (73) + : +- CometNativeColumnarToRow (73) : +- CometColumnarExchange (72) : +- * HashAggregate (71) : +- * Project (70) @@ -74,29 +74,29 @@ : : : :- BroadcastExchange (53) : : : : +- * ColumnarToRow (52) : : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : : +- * CometColumnarToRow (57) + : : : +- CometNativeColumnarToRow (57) : : : +- CometProject (56) : : : +- CometFilter (55) : : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) : : +- ReusedExchange (61) : +- BroadcastExchange (68) - : +- * CometColumnarToRow (67) + : +- CometNativeColumnarToRow (67) : +- CometProject (66) : +- CometFilter (65) : +- CometNativeScan parquet spark_catalog.default.web_site (64) :- * HashAggregate (86) - : +- * CometColumnarToRow (85) + : +- CometNativeColumnarToRow (85) : +- CometColumnarExchange (84) : +- * HashAggregate (83) : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) + : +- CometNativeColumnarToRow (81) : +- ReusedExchange (80) +- * HashAggregate (93) - +- * CometColumnarToRow (92) + +- CometNativeColumnarToRow (92) +- CometColumnarExchange (91) +- * HashAggregate (90) +- * HashAggregate (89) - +- * CometColumnarToRow (88) + +- CometNativeColumnarToRow (88) +- ReusedExchange (87) @@ -143,13 +143,13 @@ Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_s (10) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#22] -(11) BroadcastHashJoin [codegen id : 5] +(11) BroadcastHashJoin [codegen id : 3] Left keys [1]: [date_sk#7] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(12) Project [codegen id : 5] +(12) Project [codegen id : 3] Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] @@ -168,24 +168,24 @@ Condition : isnotnull(s_store_sk#23) Input [2]: [s_store_sk#23, s_store_id#24] Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#23, s_store_id#25] (17) BroadcastExchange Input [2]: [s_store_sk#23, s_store_id#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 5] +(18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [store_sk#6] Right keys [1]: [s_store_sk#23] Join type: Inner Join condition: None -(19) Project [codegen id : 5] +(19) Project [codegen id : 3] Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] -(20) HashAggregate [codegen id : 5] +(20) HashAggregate [codegen id : 3] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] Keys [1]: [s_store_id#25] Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] @@ -196,10 +196,10 @@ Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -(23) HashAggregate [codegen id : 6] +(23) HashAggregate [codegen id : 4] Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] Keys [1]: [s_store_id#25] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] @@ -214,14 +214,14 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] +(25) ColumnarToRow [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -(26) Filter [codegen id : 7] +(26) Filter [codegen id : 5] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] Condition : isnotnull(cs_catalog_page_sk#43) -(27) Project [codegen id : 7] +(27) Project [codegen id : 5] Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] @@ -233,14 +233,14 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(29) ColumnarToRow [codegen id : 8] +(29) ColumnarToRow [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -(30) Filter [codegen id : 8] +(30) Filter [codegen id : 6] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Condition : isnotnull(cr_catalog_page_sk#53) -(31) Project [codegen id : 8] +(31) Project [codegen id : 6] Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] @@ -249,13 +249,13 @@ Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_retur (33) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#63] -(34) BroadcastHashJoin [codegen id : 11] +(34) BroadcastHashJoin [codegen id : 7] Left keys [1]: [date_sk#48] Right keys [1]: [d_date_sk#63] Join type: Inner Join condition: None -(35) Project [codegen id : 11] +(35) Project [codegen id : 7] Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] @@ -274,24 +274,24 @@ Condition : isnotnull(cp_catalog_page_sk#64) Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#65, 16, true, false, true) AS cp_catalog_page_id#66] -(39) CometColumnarToRow [codegen id : 10] +(39) CometNativeColumnarToRow Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] (40) BroadcastExchange Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(41) BroadcastHashJoin [codegen id : 11] +(41) BroadcastHashJoin [codegen id : 7] Left keys [1]: [page_sk#47] Right keys [1]: [cp_catalog_page_sk#64] Join type: Inner Join condition: None -(42) Project [codegen id : 11] +(42) Project [codegen id : 7] Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] -(43) HashAggregate [codegen id : 11] +(43) HashAggregate [codegen id : 7] Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] @@ -302,10 +302,10 @@ Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(45) CometColumnarToRow [codegen id : 12] +(45) CometNativeColumnarToRow Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -(46) HashAggregate [codegen id : 12] +(46) HashAggregate [codegen id : 8] Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] Keys [1]: [cp_catalog_page_id#66] Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] @@ -320,14 +320,14 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 13] +(48) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -(49) Filter [codegen id : 13] +(49) Filter [codegen id : 9] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] Condition : isnotnull(ws_web_site_sk#84) -(50) Project [codegen id : 13] +(50) Project [codegen id : 9] Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] @@ -338,7 +338,7 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] ReadSchema: struct -(52) ColumnarToRow [codegen id : 14] +(52) ColumnarToRow [codegen id : 10] Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (53) BroadcastExchange @@ -360,16 +360,16 @@ Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND i Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(57) CometColumnarToRow +(57) CometNativeColumnarToRow Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] -(58) BroadcastHashJoin [codegen id : 15] +(58) BroadcastHashJoin [codegen id : 11] Left keys [2]: [wr_item_sk#94, wr_order_number#95] Right keys [2]: [ws_item_sk#99, ws_order_number#101] Join type: Inner Join condition: None -(59) Project [codegen id : 15] +(59) Project [codegen id : 11] Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] @@ -378,13 +378,13 @@ Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, (61) ReusedExchange [Reuses operator id: 104] Output [1]: [d_date_sk#109] -(62) BroadcastHashJoin [codegen id : 18] +(62) BroadcastHashJoin [codegen id : 12] Left keys [1]: [date_sk#89] Right keys [1]: [d_date_sk#109] Join type: Inner Join condition: None -(63) Project [codegen id : 18] +(63) Project [codegen id : 12] Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] @@ -403,24 +403,24 @@ Condition : isnotnull(web_site_sk#110) Input [2]: [web_site_sk#110, web_site_id#111] Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#111, 16, true, false, true) AS web_site_id#112] -(67) CometColumnarToRow [codegen id : 17] +(67) CometNativeColumnarToRow Input [2]: [web_site_sk#110, web_site_id#112] (68) BroadcastExchange Input [2]: [web_site_sk#110, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -(69) BroadcastHashJoin [codegen id : 18] +(69) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wsr_web_site_sk#88] Right keys [1]: [web_site_sk#110] Join type: Inner Join condition: None -(70) Project [codegen id : 18] +(70) Project [codegen id : 12] Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] -(71) HashAggregate [codegen id : 18] +(71) HashAggregate [codegen id : 12] Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] Keys [1]: [web_site_id#112] Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] @@ -431,10 +431,10 @@ Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometColumnarToRow [codegen id : 19] +(73) CometNativeColumnarToRow Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -(74) HashAggregate [codegen id : 19] +(74) HashAggregate [codegen id : 13] Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] Keys [1]: [web_site_id#112] Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] @@ -443,7 +443,7 @@ Results [5]: [web channel AS channel#125, concat(web_site, web_site_id#112) AS i (75) Union -(76) HashAggregate [codegen id : 20] +(76) HashAggregate [codegen id : 14] Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] Keys [2]: [channel#38, id#39] Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] @@ -454,10 +454,10 @@ Results [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(78) CometColumnarToRow [codegen id : 21] +(78) CometNativeColumnarToRow Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(79) HashAggregate [codegen id : 21] +(79) HashAggregate [codegen id : 15] Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Keys [2]: [channel#38, id#39] Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] @@ -467,17 +467,17 @@ Results [5]: [channel#38, id#39, cast(sum(sales#40)#142 as decimal(37,2)) AS sal (80) ReusedExchange [Reuses operator id: 77] Output [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(81) CometColumnarToRow [codegen id : 42] +(81) CometNativeColumnarToRow Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(82) HashAggregate [codegen id : 42] +(82) HashAggregate [codegen id : 30] Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Keys [2]: [channel#38, id#39] Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] Results [4]: [channel#38, sum(sales#40)#142 AS sales#148, sum(returns#41)#143 AS returns#149, sum(profit#42)#144 AS profit#150] -(83) HashAggregate [codegen id : 42] +(83) HashAggregate [codegen id : 30] Input [4]: [channel#38, sales#148, returns#149, profit#150] Keys [1]: [channel#38] Functions [3]: [partial_sum(sales#148), partial_sum(returns#149), partial_sum(profit#150)] @@ -488,10 +488,10 @@ Results [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, i Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Arguments: hashpartitioning(channel#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(85) CometColumnarToRow [codegen id : 43] +(85) CometNativeColumnarToRow Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -(86) HashAggregate [codegen id : 43] +(86) HashAggregate [codegen id : 31] Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Keys [1]: [channel#38] Functions [3]: [sum(sales#148), sum(returns#149), sum(profit#150)] @@ -501,17 +501,17 @@ Results [5]: [channel#38, null AS id#166, sum(sales#148)#163 AS sum(sales)#167, (87) ReusedExchange [Reuses operator id: 77] Output [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(88) CometColumnarToRow [codegen id : 64] +(88) CometNativeColumnarToRow Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -(89) HashAggregate [codegen id : 64] +(89) HashAggregate [codegen id : 46] Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] Keys [2]: [channel#38, id#39] Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] Results [3]: [sum(sales#40)#142 AS sales#148, sum(returns#41)#143 AS returns#149, sum(profit#42)#144 AS profit#150] -(90) HashAggregate [codegen id : 64] +(90) HashAggregate [codegen id : 46] Input [3]: [sales#148, returns#149, profit#150] Keys: [] Functions [3]: [partial_sum(sales#148), partial_sum(returns#149), partial_sum(profit#150)] @@ -522,10 +522,10 @@ Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(92) CometColumnarToRow [codegen id : 65] +(92) CometNativeColumnarToRow Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -(93) HashAggregate [codegen id : 65] +(93) HashAggregate [codegen id : 47] Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys: [] Functions [3]: [sum(sales#148), sum(returns#149), sum(profit#150)] @@ -534,7 +534,7 @@ Results [5]: [null AS channel#185, null AS id#186, sum(sales#148)#182 AS sum(sal (94) Union -(95) HashAggregate [codegen id : 66] +(95) HashAggregate [codegen id : 48] Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Functions: [] @@ -554,14 +554,14 @@ Functions: [] Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#145,returns#146,profit#147]), [channel#38, id#39, sales#145, returns#146, profit#147], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] -(99) CometColumnarToRow [codegen id : 67] +(99) CometNativeColumnarToRow Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (104) -+- * CometColumnarToRow (103) ++- CometNativeColumnarToRow (103) +- CometProject (102) +- CometFilter (101) +- CometNativeScan parquet spark_catalog.default.date_dim (100) @@ -582,7 +582,7 @@ Condition : (((isnotnull(d_date#190) AND (d_date#190 >= 1998-08-04)) AND (d_date Input [2]: [d_date_sk#22, d_date#190] Arguments: [d_date_sk#22], [d_date_sk#22] -(103) CometColumnarToRow [codegen id : 1] +(103) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (104) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt index 63ea8e5ea8..ec2eef025a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt @@ -1,16 +1,16 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,17 +34,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -63,17 +63,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -92,31 +92,31 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_site :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -130,7 +130,7 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -140,17 +140,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -169,17 +169,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -198,31 +198,31 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_site +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -236,7 +236,7 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -246,17 +246,17 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -275,17 +275,17 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.catalog_page +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -304,17 +304,17 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/simplified.txt index dc9c9b1ce7..ae0afc7f07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/simplified.txt @@ -1,167 +1,157 @@ -WholeStageCodegen (67) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (66) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (21) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (20) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (48) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (15) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (14) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_store_id] #3 + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #8 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 + Union + WholeStageCodegen (1) + Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] + Filter [ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + WholeStageCodegen (2) + Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] + Filter [sr_store_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (43) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #11 - WholeStageCodegen (42) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (65) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (64) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow + CometNativeColumnarToRow + CometColumnarExchange [cp_catalog_page_id] #6 + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (5) + Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] + Filter [cs_catalog_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + WholeStageCodegen (6) + Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] + Filter [cr_catalog_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (13) + HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometNativeColumnarToRow + CometColumnarExchange [web_site_id] #8 + WholeStageCodegen (12) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (9) + Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] + Filter [ws_web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + WholeStageCodegen (11) + Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + CometNativeColumnarToRow + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #10 + CometNativeColumnarToRow + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (31) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel] #11 + WholeStageCodegen (30) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (47) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #12 + WholeStageCodegen (46) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt index 306485ca63..a905f9b01a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt @@ -483,7 +483,7 @@ Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) @@ -504,7 +504,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt index 35b86dbff5..24167a2372 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,7 +197,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt index c0b236a57a..0084fda697 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt @@ -23,12 +23,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 306485ca63..a905f9b01a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -483,7 +483,7 @@ Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (89) -+- * CometColumnarToRow (88) ++- CometNativeColumnarToRow (88) +- CometProject (87) +- CometFilter (86) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) @@ -504,7 +504,7 @@ Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#2 Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(88) CometColumnarToRow [codegen id : 1] +(88) CometNativeColumnarToRow Input [1]: [d_date_sk#22] (89) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/extended.txt index 35b86dbff5..24167a2372 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,7 +109,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -197,7 +197,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 c0b236a57a..0084fda697 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 @@ -23,12 +23,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/explain.txt index e0b30c6833..14ddc524ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (40) +- * Filter (39) +- * HashAggregate (38) - +- * CometColumnarToRow (37) + +- CometNativeColumnarToRow (37) +- CometColumnarExchange (36) +- * HashAggregate (35) +- * Project (34) @@ -11,7 +11,7 @@ TakeOrderedAndProject (40) : +- * BroadcastHashJoin Inner BuildRight (17) : :- * Project (15) : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) + : : :- CometNativeColumnarToRow (9) : : : +- CometProject (8) : : : +- CometBroadcastHashJoin (7) : : : :- CometProject (3) @@ -26,7 +26,7 @@ TakeOrderedAndProject (40) : : +- Scan parquet spark_catalog.default.store_sales (10) : +- ReusedExchange (16) +- BroadcastExchange (32) - +- * CometColumnarToRow (31) + +- CometNativeColumnarToRow (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (20) @@ -80,7 +80,7 @@ Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] -(9) CometColumnarToRow [codegen id : 4] +(9) CometNativeColumnarToRow Input [2]: [ca_state#3, c_customer_sk#4] (10) Scan parquet spark_catalog.default.store_sales @@ -102,26 +102,26 @@ Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] -(14) BroadcastHashJoin [codegen id : 4] +(14) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#4] Right keys [1]: [ss_customer_sk#7] Join type: Inner Join condition: None -(15) Project [codegen id : 4] +(15) Project [codegen id : 2] Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] (16) ReusedExchange [Reuses operator id: 45] Output [1]: [d_date_sk#10] -(17) BroadcastHashJoin [codegen id : 4] +(17) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#8] Right keys [1]: [d_date_sk#10] Join type: Inner Join condition: None -(18) Project [codegen id : 4] +(18) Project [codegen id : 2] Output [2]: [ca_state#3, ss_item_sk#6] Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] @@ -181,24 +181,24 @@ Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCod Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#17, i_category#14] Arguments: [i_item_sk#11], [i_item_sk#11] -(31) CometColumnarToRow [codegen id : 3] +(31) CometNativeColumnarToRow Input [1]: [i_item_sk#11] (32) BroadcastExchange Input [1]: [i_item_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) BroadcastHashJoin [codegen id : 4] +(33) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#6] Right keys [1]: [i_item_sk#11] Join type: Inner Join condition: None -(34) Project [codegen id : 4] +(34) Project [codegen id : 2] Output [1]: [ca_state#3] Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] -(35) HashAggregate [codegen id : 4] +(35) HashAggregate [codegen id : 2] Input [1]: [ca_state#3] Keys [1]: [ca_state#3] Functions [1]: [partial_count(1)] @@ -209,17 +209,17 @@ Results [2]: [ca_state#3, count#19] Input [2]: [ca_state#3, count#19] Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometColumnarToRow [codegen id : 5] +(37) CometNativeColumnarToRow Input [2]: [ca_state#3, count#19] -(38) HashAggregate [codegen id : 5] +(38) HashAggregate [codegen id : 3] Input [2]: [ca_state#3, count#19] Keys [1]: [ca_state#3] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#20] Results [3]: [ca_state#3 AS state#21, count(1)#20 AS cnt#22, ca_state#3] -(39) Filter [codegen id : 5] +(39) Filter [codegen id : 3] Input [3]: [state#21, cnt#22, ca_state#3] Condition : (cnt#22 >= 10) @@ -231,7 +231,7 @@ Arguments: 100, [cnt#22 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#21, Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometNativeScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#23] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#23] Keys [1]: [d_month_seq#23] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#23] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt index fe3b28849a..f2cdb50e03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Filter +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -10,7 +10,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometProject @@ -25,11 +25,11 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -38,11 +38,11 @@ TakeOrderedAndProject : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : : +- Subquery - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -51,7 +51,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/simplified.txt index 76dd43f9c7..9114fcaf65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/simplified.txt @@ -1,11 +1,11 @@ TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (5) + WholeStageCodegen (3) Filter [cnt] HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) + WholeStageCodegen (2) HashAggregate [ca_state] [count,count] Project [ca_state] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -13,8 +13,8 @@ TakeOrderedAndProject [cnt,ca_state,state] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [ca_state,ss_item_sk,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [ca_state,c_customer_sk] CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] CometProject [ca_state] [ca_address_sk,ca_state] @@ -32,38 +32,32 @@ TakeOrderedAndProject [cnt,ca_state,state] Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter ReusedExchange [d_date_sk] #4 InputAdapter BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] + CometNativeColumnarToRow + CometProject [i_item_sk] + CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] + CometFilter [i_item_sk,i_current_price,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + CometBroadcastExchange [avg(i_current_price),i_category] #7 + CometFilter [avg(i_current_price),i_category] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] + CometExchange [i_category] #8 + CometHashAggregate [i_current_price] [i_category,sum,count] + CometProject [i_category] [i_current_price,i_category] + CometFilter [i_current_price,i_category] + CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/explain.txt index f6395ec27c..e39af3acfc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#22, cnt#23] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#11] Keys [1]: [d_month_seq#11] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#11] Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt index f5b69fc6cf..20115b61c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt @@ -23,11 +23,11 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/simplified.txt index 2978e30c1c..fd51229f24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/simplified.txt @@ -25,22 +25,18 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index f6395ec27c..e39af3acfc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -231,7 +231,7 @@ Input [2]: [state#22, cnt#23] Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (45) -+- * CometColumnarToRow (44) ++- CometNativeColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-su Input [2]: [d_date_sk#10, d_month_seq#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(44) CometColumnarToRow [codegen id : 1] +(44) CometNativeColumnarToRow Input [1]: [d_date_sk#10] (45) BroadcastExchange @@ -260,7 +260,7 @@ Input [1]: [d_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) +CometNativeColumnarToRow (52) +- CometHashAggregate (51) +- CometExchange (50) +- CometHashAggregate (49) @@ -298,7 +298,7 @@ Input [1]: [d_month_seq#11] Keys [1]: [d_month_seq#11] Functions: [] -(52) CometColumnarToRow [codegen id : 1] +(52) CometNativeColumnarToRow Input [1]: [d_month_seq#11] Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/extended.txt index f5b69fc6cf..20115b61c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/extended.txt @@ -23,11 +23,11 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate 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 2978e30c1c..fd51229f24 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 @@ -25,22 +25,18 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + Subquery #2 + CometNativeColumnarToRow + CometHashAggregate [d_month_seq] + CometExchange [d_month_seq] #5 + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #6 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/explain.txt index 135549db3d..322821e984 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (180) +CometNativeColumnarToRow (180) +- CometSort (179) +- CometExchange (178) +- CometProject (177) @@ -48,7 +48,7 @@ : : : : : : : : : : : : : : : : : : +- * Filter (3) : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (8) : : : : : : : : : : : : : : : : : +- CometProject (7) : : : : : : : : : : : : : : : : : +- CometFilter (6) : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) @@ -153,7 +153,7 @@ : : : : : : : : : : : : : : : : : +- * Filter (108) : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow (113) : : : : : : : : : : : : : : : : +- CometProject (112) : : : : : : : : : : : : : : : : +- CometFilter (111) : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) @@ -215,7 +215,7 @@ Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] -(8) CometColumnarToRow +(8) CometNativeColumnarToRow Input [2]: [sr_item_sk#14, sr_ticket_number#15] (9) BroadcastHashJoin [codegen id : 2] @@ -694,7 +694,7 @@ Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] -(113) CometColumnarToRow +(113) CometNativeColumnarToRow Input [2]: [sr_item_sk#111, sr_ticket_number#112] (114) BroadcastHashJoin [codegen id : 4] @@ -971,14 +971,14 @@ Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC 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: [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 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] -(180) CometColumnarToRow [codegen id : 5] +(180) CometNativeColumnarToRow 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometNativeScan parquet spark_catalog.default.date_dim (181) @@ -994,7 +994,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (184) BroadcastExchange @@ -1003,7 +1003,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (188) -+- * CometColumnarToRow (187) ++- CometNativeColumnarToRow (187) +- CometFilter (186) +- CometNativeScan parquet spark_catalog.default.date_dim (185) @@ -1019,7 +1019,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(187) CometColumnarToRow [codegen id : 1] +(187) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (188) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt index 6e3f37148b..98db64e5b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometExchange +- CometProject @@ -49,10 +49,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -171,10 +171,10 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/simplified.txt index d933995ffc..c902a59f78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/simplified.txt @@ -1,206 +1,200 @@ -WholeStageCodegen (5) - 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] - 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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_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] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] +CometNativeColumnarToRow + 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] + 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #3 + WholeStageCodegen (2) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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_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] + CometExchange [cs_item_sk,cs_order_number] #7 + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometExchange [cr_item_sk,cr_order_number] #8 + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #9 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 + CometProject [s_zip] [s_store_sk,s_store_name,s_zip] + CometFilter [s_store_sk,s_store_name,s_zip] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + CometBroadcastExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastExchange [d_date_sk,d_year] #12 + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [d_date_sk,d_year] #12 + CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 + CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + CometBroadcastExchange [p_promo_sk] #14 + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 + CometFilter [hd_demo_sk,hd_income_band_sk] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 + CometBroadcastExchange [ib_income_band_sk] #17 + CometFilter [ib_income_band_sk] + CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ib_income_band_sk] #17 + CometBroadcastExchange [i_item_sk,i_product_name] #18 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_current_price,i_color,i_product_name] + CometNativeScan 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 [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] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] + CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk] #20 + WholeStageCodegen (4) + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + BroadcastExchange #21 + WholeStageCodegen (3) + Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #22 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + CometNativeColumnarToRow + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + CometSort [cs_item_sk] + CometProject [cs_item_sk] + CometFilter [cs_item_sk,sale,refund] + CometHashAggregate [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] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + ReusedExchange [s_store_sk,s_store_name,s_zip] #10 + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + ReusedExchange [d_date_sk,d_year] #12 ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + ReusedExchange [cd_demo_sk,cd_marital_status] #13 + ReusedExchange [p_promo_sk] #14 ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan 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 [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] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk,cs_item_sk] - CometSort [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [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] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 + ReusedExchange [ib_income_band_sk] #17 + ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/explain.txt index 077daeca3a..fe232d03f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/explain.txt @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/simplified.txt index 46fe063430..f1bd37b320 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/simplified.txt @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] 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 077daeca3a..fe232d03f0 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 @@ -960,7 +960,7 @@ Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 BroadcastExchange (180) -+- * CometColumnarToRow (179) ++- CometNativeColumnarToRow (179) +- CometFilter (178) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) @@ -976,7 +976,7 @@ ReadSchema: struct Input [2]: [d_date_sk#32, d_year#33] Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) -(179) CometColumnarToRow [codegen id : 1] +(179) CometNativeColumnarToRow Input [2]: [d_date_sk#32, d_year#33] (180) BroadcastExchange @@ -985,7 +985,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 BroadcastExchange (184) -+- * CometColumnarToRow (183) ++- CometNativeColumnarToRow (183) +- CometFilter (182) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) @@ -1001,7 +1001,7 @@ ReadSchema: struct Input [2]: [d_date_sk#122, d_year#123] Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) -(183) CometColumnarToRow [codegen id : 1] +(183) CometNativeColumnarToRow Input [2]: [d_date_sk#122, d_year#123] (184) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/extended.txt index 059acab385..75884c3f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject 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 46fe063430..f1bd37b320 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 @@ -50,11 +50,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] @@ -160,11 +158,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometProject [sr_item_sk,sr_ticket_number] CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/explain.txt index 674c7c227b..9387df7e7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/explain.txt @@ -2,12 +2,12 @@ TakeOrderedAndProject (87) +- * Filter (86) +- Window (85) - +- * CometColumnarToRow (84) + +- CometNativeColumnarToRow (84) +- CometSort (83) +- CometColumnarExchange (82) +- Union (81) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * Project (20) @@ -21,70 +21,70 @@ TakeOrderedAndProject (87) : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : +- ReusedExchange (4) : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) + : : +- CometNativeColumnarToRow (10) : : +- CometProject (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) + : +- CometNativeColumnarToRow (17) : +- CometProject (16) : +- CometFilter (15) : +- CometNativeScan parquet spark_catalog.default.item (14) :- * HashAggregate (31) - : +- * CometColumnarToRow (30) + : +- CometNativeColumnarToRow (30) : +- CometColumnarExchange (29) : +- * HashAggregate (28) : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) + : +- CometNativeColumnarToRow (26) : +- ReusedExchange (25) :- * HashAggregate (38) - : +- * CometColumnarToRow (37) + : +- CometNativeColumnarToRow (37) : +- CometColumnarExchange (36) : +- * HashAggregate (35) : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) + : +- CometNativeColumnarToRow (33) : +- ReusedExchange (32) :- * HashAggregate (45) - : +- * CometColumnarToRow (44) + : +- CometNativeColumnarToRow (44) : +- CometColumnarExchange (43) : +- * HashAggregate (42) : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) + : +- CometNativeColumnarToRow (40) : +- ReusedExchange (39) :- * HashAggregate (52) - : +- * CometColumnarToRow (51) + : +- CometNativeColumnarToRow (51) : +- CometColumnarExchange (50) : +- * HashAggregate (49) : +- * HashAggregate (48) - : +- * CometColumnarToRow (47) + : +- CometNativeColumnarToRow (47) : +- ReusedExchange (46) :- * HashAggregate (59) - : +- * CometColumnarToRow (58) + : +- CometNativeColumnarToRow (58) : +- CometColumnarExchange (57) : +- * HashAggregate (56) : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) + : +- CometNativeColumnarToRow (54) : +- ReusedExchange (53) :- * HashAggregate (66) - : +- * CometColumnarToRow (65) + : +- CometNativeColumnarToRow (65) : +- CometColumnarExchange (64) : +- * HashAggregate (63) : +- * HashAggregate (62) - : +- * CometColumnarToRow (61) + : +- CometNativeColumnarToRow (61) : +- ReusedExchange (60) :- * HashAggregate (73) - : +- * CometColumnarToRow (72) + : +- CometNativeColumnarToRow (72) : +- CometColumnarExchange (71) : +- * HashAggregate (70) : +- * HashAggregate (69) - : +- * CometColumnarToRow (68) + : +- CometNativeColumnarToRow (68) : +- ReusedExchange (67) +- * HashAggregate (80) - +- * CometColumnarToRow (79) + +- CometNativeColumnarToRow (79) +- CometColumnarExchange (78) +- * HashAggregate (77) +- * HashAggregate (76) - +- * CometColumnarToRow (75) + +- CometNativeColumnarToRow (75) +- ReusedExchange (74) @@ -96,23 +96,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 4] +(2) ColumnarToRow [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -(3) Filter [codegen id : 4] +(3) Filter [codegen id : 1] Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) (4) ReusedExchange [Reuses operator id: 92] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(5) BroadcastHashJoin [codegen id : 4] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#5] Right keys [1]: [d_date_sk#7] Join type: Inner Join condition: None -(6) Project [codegen id : 4] +(6) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] @@ -131,20 +131,20 @@ Condition : isnotnull(s_store_sk#11) Input [2]: [s_store_sk#11, s_store_id#12] Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [2]: [s_store_sk#11, s_store_id#13] (11) BroadcastExchange Input [2]: [s_store_sk#11, s_store_id#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 4] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#2] Right keys [1]: [s_store_sk#11] Join type: Inner Join condition: None -(13) Project [codegen id : 4] +(13) Project [codegen id : 1] Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] @@ -163,24 +163,24 @@ Condition : isnotnull(i_item_sk#14) Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] -(17) CometColumnarToRow [codegen id : 3] +(17) CometNativeColumnarToRow Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] (18) BroadcastExchange Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] -(19) BroadcastHashJoin [codegen id : 4] +(19) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#14] Join type: Inner Join condition: None -(20) Project [codegen id : 4] +(20) Project [codegen id : 1] Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -(21) HashAggregate [codegen id : 4] +(21) HashAggregate [codegen id : 1] Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -191,10 +191,10 @@ Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year# Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 5] +(23) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -(24) HashAggregate [codegen id : 5] +(24) HashAggregate [codegen id : 2] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] @@ -204,17 +204,17 @@ Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand# (25) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] -(26) CometColumnarToRow [codegen id : 10] +(26) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] -(27) HashAggregate [codegen id : 10] +(27) HashAggregate [codegen id : 4] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27] Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27 AS sumsales#44] -(28) HashAggregate [codegen id : 10] +(28) HashAggregate [codegen id : 4] Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sumsales#44] Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] Functions [1]: [partial_sum(sumsales#44)] @@ -225,10 +225,10 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#3 Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 11] +(30) CometNativeColumnarToRow Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -(31) HashAggregate [codegen id : 11] +(31) HashAggregate [codegen id : 5] Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] Functions [1]: [sum(sumsales#44)] @@ -238,17 +238,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#3 (32) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] -(33) CometColumnarToRow [codegen id : 16] +(33) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] -(34) HashAggregate [codegen id : 16] +(34) HashAggregate [codegen id : 7] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27] Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27 AS sumsales#44] -(35) HashAggregate [codegen id : 16] +(35) HashAggregate [codegen id : 7] Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sumsales#44] Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] Functions [1]: [partial_sum(sumsales#44)] @@ -259,10 +259,10 @@ Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#5 Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#61, isEmpty#62] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(37) CometColumnarToRow [codegen id : 17] +(37) CometNativeColumnarToRow Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#61, isEmpty#62] -(38) HashAggregate [codegen id : 17] +(38) HashAggregate [codegen id : 8] Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#61, isEmpty#62] Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] Functions [1]: [sum(sumsales#44)] @@ -272,17 +272,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#5 (39) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, d_qoy#68, d_moy#69, s_store_id#13, sum#70, isEmpty#71] -(40) CometColumnarToRow [codegen id : 22] +(40) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, d_qoy#68, d_moy#69, s_store_id#13, sum#70, isEmpty#71] -(41) HashAggregate [codegen id : 22] +(41) HashAggregate [codegen id : 10] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, d_qoy#68, d_moy#69, s_store_id#13, sum#70, isEmpty#71] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, d_qoy#68, d_moy#69, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#72 * cast(ss_quantity#73 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#72 * cast(ss_quantity#73 as decimal(10,0))), 0.00))#27] Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sum(coalesce((ss_sales_price#72 * cast(ss_quantity#73 as decimal(10,0))), 0.00))#27 AS sumsales#44] -(42) HashAggregate [codegen id : 22] +(42) HashAggregate [codegen id : 10] Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sumsales#44] Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67] Functions [1]: [partial_sum(sumsales#44)] @@ -293,10 +293,10 @@ Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#6 Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sum#76, isEmpty#77] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(44) CometColumnarToRow [codegen id : 23] +(44) CometNativeColumnarToRow Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sum#76, isEmpty#77] -(45) HashAggregate [codegen id : 23] +(45) HashAggregate [codegen id : 11] Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sum#76, isEmpty#77] Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67] Functions [1]: [sum(sumsales#44)] @@ -306,17 +306,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#6 (46) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#83, d_qoy#84, d_moy#85, s_store_id#13, sum#86, isEmpty#87] -(47) CometColumnarToRow [codegen id : 28] +(47) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#83, d_qoy#84, d_moy#85, s_store_id#13, sum#86, isEmpty#87] -(48) HashAggregate [codegen id : 28] +(48) HashAggregate [codegen id : 13] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#83, d_qoy#84, d_moy#85, s_store_id#13, sum#86, isEmpty#87] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#83, d_qoy#84, d_moy#85, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))#27] Results [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))#27 AS sumsales#44] -(49) HashAggregate [codegen id : 28] +(49) HashAggregate [codegen id : 13] Input [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sumsales#44] Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] Functions [1]: [partial_sum(sumsales#44)] @@ -327,10 +327,10 @@ Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#92, Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#92, isEmpty#93] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) CometColumnarToRow [codegen id : 29] +(51) CometNativeColumnarToRow Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#92, isEmpty#93] -(52) HashAggregate [codegen id : 29] +(52) HashAggregate [codegen id : 14] Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#92, isEmpty#93] Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] Functions [1]: [sum(sumsales#44)] @@ -340,17 +340,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, null AS (53) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#100, d_qoy#101, d_moy#102, s_store_id#13, sum#103, isEmpty#104] -(54) CometColumnarToRow [codegen id : 34] +(54) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#100, d_qoy#101, d_moy#102, s_store_id#13, sum#103, isEmpty#104] -(55) HashAggregate [codegen id : 34] +(55) HashAggregate [codegen id : 16] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#100, d_qoy#101, d_moy#102, s_store_id#13, sum#103, isEmpty#104] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#100, d_qoy#101, d_moy#102, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#27] Results [4]: [i_category#21, i_class#20, i_brand#19, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#27 AS sumsales#44] -(56) HashAggregate [codegen id : 34] +(56) HashAggregate [codegen id : 16] Input [4]: [i_category#21, i_class#20, i_brand#19, sumsales#44] Keys [3]: [i_category#21, i_class#20, i_brand#19] Functions [1]: [partial_sum(sumsales#44)] @@ -361,10 +361,10 @@ Results [5]: [i_category#21, i_class#20, i_brand#19, sum#109, isEmpty#110] Input [5]: [i_category#21, i_class#20, i_brand#19, sum#109, isEmpty#110] Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(58) CometColumnarToRow [codegen id : 35] +(58) CometNativeColumnarToRow Input [5]: [i_category#21, i_class#20, i_brand#19, sum#109, isEmpty#110] -(59) HashAggregate [codegen id : 35] +(59) HashAggregate [codegen id : 17] Input [5]: [i_category#21, i_class#20, i_brand#19, sum#109, isEmpty#110] Keys [3]: [i_category#21, i_class#20, i_brand#19] Functions [1]: [sum(sumsales#44)] @@ -374,17 +374,17 @@ Results [9]: [i_category#21, i_class#20, i_brand#19, null AS i_product_name#112, (60) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#118, d_qoy#119, d_moy#120, s_store_id#13, sum#121, isEmpty#122] -(61) CometColumnarToRow [codegen id : 40] +(61) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#118, d_qoy#119, d_moy#120, s_store_id#13, sum#121, isEmpty#122] -(62) HashAggregate [codegen id : 40] +(62) HashAggregate [codegen id : 19] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#118, d_qoy#119, d_moy#120, s_store_id#13, sum#121, isEmpty#122] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#118, d_qoy#119, d_moy#120, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#123 * cast(ss_quantity#124 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#123 * cast(ss_quantity#124 as decimal(10,0))), 0.00))#27] Results [3]: [i_category#21, i_class#20, sum(coalesce((ss_sales_price#123 * cast(ss_quantity#124 as decimal(10,0))), 0.00))#27 AS sumsales#44] -(63) HashAggregate [codegen id : 40] +(63) HashAggregate [codegen id : 19] Input [3]: [i_category#21, i_class#20, sumsales#44] Keys [2]: [i_category#21, i_class#20] Functions [1]: [partial_sum(sumsales#44)] @@ -395,10 +395,10 @@ Results [4]: [i_category#21, i_class#20, sum#127, isEmpty#128] Input [4]: [i_category#21, i_class#20, sum#127, isEmpty#128] Arguments: hashpartitioning(i_category#21, i_class#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(65) CometColumnarToRow [codegen id : 41] +(65) CometNativeColumnarToRow Input [4]: [i_category#21, i_class#20, sum#127, isEmpty#128] -(66) HashAggregate [codegen id : 41] +(66) HashAggregate [codegen id : 20] Input [4]: [i_category#21, i_class#20, sum#127, isEmpty#128] Keys [2]: [i_category#21, i_class#20] Functions [1]: [sum(sumsales#44)] @@ -408,17 +408,17 @@ Results [9]: [i_category#21, i_class#20, null AS i_brand#130, null AS i_product_ (67) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#137, d_qoy#138, d_moy#139, s_store_id#13, sum#140, isEmpty#141] -(68) CometColumnarToRow [codegen id : 46] +(68) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#137, d_qoy#138, d_moy#139, s_store_id#13, sum#140, isEmpty#141] -(69) HashAggregate [codegen id : 46] +(69) HashAggregate [codegen id : 22] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#137, d_qoy#138, d_moy#139, s_store_id#13, sum#140, isEmpty#141] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#137, d_qoy#138, d_moy#139, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#142 * cast(ss_quantity#143 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#142 * cast(ss_quantity#143 as decimal(10,0))), 0.00))#27] Results [2]: [i_category#21, sum(coalesce((ss_sales_price#142 * cast(ss_quantity#143 as decimal(10,0))), 0.00))#27 AS sumsales#44] -(70) HashAggregate [codegen id : 46] +(70) HashAggregate [codegen id : 22] Input [2]: [i_category#21, sumsales#44] Keys [1]: [i_category#21] Functions [1]: [partial_sum(sumsales#44)] @@ -429,10 +429,10 @@ Results [3]: [i_category#21, sum#146, isEmpty#147] Input [3]: [i_category#21, sum#146, isEmpty#147] Arguments: hashpartitioning(i_category#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(72) CometColumnarToRow [codegen id : 47] +(72) CometNativeColumnarToRow Input [3]: [i_category#21, sum#146, isEmpty#147] -(73) HashAggregate [codegen id : 47] +(73) HashAggregate [codegen id : 23] Input [3]: [i_category#21, sum#146, isEmpty#147] Keys [1]: [i_category#21] Functions [1]: [sum(sumsales#44)] @@ -442,17 +442,17 @@ Results [9]: [i_category#21, null AS i_class#149, null AS i_brand#150, null AS i (74) ReusedExchange [Reuses operator id: 22] Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#157, d_qoy#158, d_moy#159, s_store_id#13, sum#160, isEmpty#161] -(75) CometColumnarToRow [codegen id : 52] +(75) CometNativeColumnarToRow Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#157, d_qoy#158, d_moy#159, s_store_id#13, sum#160, isEmpty#161] -(76) HashAggregate [codegen id : 52] +(76) HashAggregate [codegen id : 25] Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#157, d_qoy#158, d_moy#159, s_store_id#13, sum#160, isEmpty#161] Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#157, d_qoy#158, d_moy#159, s_store_id#13] Functions [1]: [sum(coalesce((ss_sales_price#162 * cast(ss_quantity#163 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#162 * cast(ss_quantity#163 as decimal(10,0))), 0.00))#27] Results [1]: [sum(coalesce((ss_sales_price#162 * cast(ss_quantity#163 as decimal(10,0))), 0.00))#27 AS sumsales#44] -(77) HashAggregate [codegen id : 52] +(77) HashAggregate [codegen id : 25] Input [1]: [sumsales#44] Keys: [] Functions [1]: [partial_sum(sumsales#44)] @@ -463,10 +463,10 @@ Results [2]: [sum#166, isEmpty#167] Input [2]: [sum#166, isEmpty#167] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(79) CometColumnarToRow [codegen id : 53] +(79) CometNativeColumnarToRow Input [2]: [sum#166, isEmpty#167] -(80) HashAggregate [codegen id : 53] +(80) HashAggregate [codegen id : 26] Input [2]: [sum#166, isEmpty#167] Keys: [] Functions [1]: [sum(sumsales#44)] @@ -483,14 +483,14 @@ Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumna Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] -(84) CometColumnarToRow [codegen id : 54] +(84) CometNativeColumnarToRow Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] (85) Window Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#178], [i_category#28], [sumsales#36 DESC NULLS LAST] -(86) Filter [codegen id : 55] +(86) Filter [codegen id : 27] Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#178] Condition : (rk#178 <= 100) @@ -502,7 +502,7 @@ Arguments: 100, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (92) -+- * CometColumnarToRow (91) ++- CometNativeColumnarToRow (91) +- CometProject (90) +- CometFilter (89) +- CometNativeScan parquet spark_catalog.default.date_dim (88) @@ -523,7 +523,7 @@ Condition : (((isnotnull(d_month_seq#179) AND (d_month_seq#179 >= 1212)) AND (d_ Input [5]: [d_date_sk#7, d_month_seq#179, d_year#8, d_moy#9, d_qoy#10] Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -(91) CometColumnarToRow [codegen id : 1] +(91) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] (92) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt index a2ddc934bb..84fdb89cb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt @@ -1,12 +1,12 @@ TakeOrderedAndProject +- Filter +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -20,31 +20,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -58,31 +58,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -96,31 +96,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -134,31 +134,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -172,31 +172,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -210,31 +210,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -248,31 +248,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -286,31 +286,31 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -324,22 +324,22 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/simplified.txt index 4bc3f849c5..d2a6ba7a8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/simplified.txt @@ -1,142 +1,134 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (55) + WholeStageCodegen (27) Filter [rk] InputAdapter Window [sumsales,i_category] - WholeStageCodegen (54) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + Union + WholeStageCodegen (2) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk,ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - 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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #11 - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #12 - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + 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] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (8) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 + WholeStageCodegen (7) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (11) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 + WholeStageCodegen (10) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (14) + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 + WholeStageCodegen (13) + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class,i_brand] #10 + WholeStageCodegen (16) + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (20) + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class] #11 + WholeStageCodegen (19) + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (23) + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category] #12 + WholeStageCodegen (22) + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (26) + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #13 + WholeStageCodegen (25) + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + 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/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt index ac04f86f96..011498d5db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt @@ -395,7 +395,7 @@ Arguments: 100, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (75) -+- * CometColumnarToRow (74) ++- CometNativeColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) @@ -416,7 +416,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(74) CometColumnarToRow [codegen id : 1] +(74) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt index 1735511c2a..c5c2f024b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,7 +50,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -82,7 +82,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -114,7 +114,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -146,7 +146,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -178,7 +178,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -210,7 +210,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -242,7 +242,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -274,7 +274,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt index 5ca258aeca..a0d899fbdc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt @@ -22,12 +22,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index ac04f86f96..011498d5db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -395,7 +395,7 @@ Arguments: 100, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_b Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (75) -+- * CometColumnarToRow (74) ++- CometNativeColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) @@ -416,7 +416,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(74) CometColumnarToRow [codegen id : 1] +(74) CometNativeColumnarToRow Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt index 1735511c2a..c5c2f024b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -50,7 +50,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -82,7 +82,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -114,7 +114,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -146,7 +146,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -178,7 +178,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -210,7 +210,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -242,7 +242,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -274,7 +274,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 5ca258aeca..a0d899fbdc 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 @@ -22,12 +22,10 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometNativeColumnarToRow + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/explain.txt index 7afb402b0b..77915922eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (64) +- * Project (63) +- Window (62) - +- * CometColumnarToRow (61) + +- CometNativeColumnarToRow (61) +- CometSort (60) +- CometExchange (59) +- CometHashAggregate (58) @@ -10,7 +10,7 @@ TakeOrderedAndProject (64) +- * HashAggregate (56) +- Union (55) :- * HashAggregate (40) - : +- * CometColumnarToRow (39) + : +- CometNativeColumnarToRow (39) : +- CometColumnarExchange (38) : +- * HashAggregate (37) : +- * Project (36) @@ -24,7 +24,7 @@ TakeOrderedAndProject (64) : +- BroadcastExchange (34) : +- * Project (33) : +- * BroadcastHashJoin LeftSemi BuildRight (32) - : :- * CometColumnarToRow (9) + : :- CometNativeColumnarToRow (9) : : +- CometFilter (8) : : +- CometNativeScan parquet spark_catalog.default.store (7) : +- BroadcastExchange (31) @@ -33,7 +33,7 @@ TakeOrderedAndProject (64) : +- Window (28) : +- * Sort (27) : +- * HashAggregate (26) - : +- * CometColumnarToRow (25) + : +- CometNativeColumnarToRow (25) : +- CometColumnarExchange (24) : +- * HashAggregate (23) : +- * Project (22) @@ -44,24 +44,24 @@ TakeOrderedAndProject (64) : : : +- * ColumnarToRow (11) : : : +- Scan parquet spark_catalog.default.store_sales (10) : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) + : : +- CometNativeColumnarToRow (16) : : +- CometProject (15) : : +- CometFilter (14) : : +- CometNativeScan parquet spark_catalog.default.store (13) : +- ReusedExchange (20) :- * HashAggregate (47) - : +- * CometColumnarToRow (46) + : +- CometNativeColumnarToRow (46) : +- CometColumnarExchange (45) : +- * HashAggregate (44) : +- * HashAggregate (43) - : +- * CometColumnarToRow (42) + : +- CometNativeColumnarToRow (42) : +- ReusedExchange (41) +- * HashAggregate (54) - +- * CometColumnarToRow (53) + +- CometNativeColumnarToRow (53) +- CometColumnarExchange (52) +- * HashAggregate (51) +- * HashAggregate (50) - +- * CometColumnarToRow (49) + +- CometNativeColumnarToRow (49) +- ReusedExchange (48) @@ -73,23 +73,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 8] +(2) ColumnarToRow [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 8] +(3) Filter [codegen id : 5] Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 69] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 8] +(5) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 8] +(6) Project [codegen id : 5] Output [2]: [ss_store_sk#1, ss_net_profit#2] Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] @@ -104,7 +104,7 @@ ReadSchema: struct Input [3]: [s_store_sk#6, s_county#7, s_state#8] Condition : isnotnull(s_store_sk#6) -(9) CometColumnarToRow [codegen id : 7] +(9) CometNativeColumnarToRow Input [3]: [s_store_sk#6, s_county#7, s_state#8] (10) Scan parquet spark_catalog.default.store_sales @@ -115,10 +115,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] +(11) ColumnarToRow [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -(12) Filter [codegen id : 4] +(12) Filter [codegen id : 1] Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] Condition : isnotnull(ss_store_sk#9) @@ -137,37 +137,37 @@ Condition : isnotnull(s_store_sk#12) Input [2]: [s_store_sk#12, s_state#13] Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#14] -(16) CometColumnarToRow [codegen id : 2] +(16) CometNativeColumnarToRow Input [2]: [s_store_sk#12, s_state#14] (17) BroadcastExchange Input [2]: [s_store_sk#12, s_state#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(18) BroadcastHashJoin [codegen id : 4] +(18) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#9] Right keys [1]: [s_store_sk#12] Join type: Inner Join condition: None -(19) Project [codegen id : 4] +(19) Project [codegen id : 1] Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] (20) ReusedExchange [Reuses operator id: 69] Output [1]: [d_date_sk#15] -(21) BroadcastHashJoin [codegen id : 4] +(21) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#11] Right keys [1]: [d_date_sk#15] Join type: Inner Join condition: None -(22) Project [codegen id : 4] +(22) Project [codegen id : 1] Output [2]: [ss_net_profit#10, s_state#14] Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] -(23) HashAggregate [codegen id : 4] +(23) HashAggregate [codegen id : 1] Input [2]: [ss_net_profit#10, s_state#14] Keys [1]: [s_state#14] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] @@ -178,17 +178,17 @@ Results [2]: [s_state#14, sum#17] Input [2]: [s_state#14, sum#17] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(25) CometColumnarToRow [codegen id : 5] +(25) CometNativeColumnarToRow Input [2]: [s_state#14, sum#17] -(26) HashAggregate [codegen id : 5] +(26) HashAggregate [codegen id : 2] Input [2]: [s_state#14, sum#17] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] -(27) Sort [codegen id : 5] +(27) Sort [codegen id : 2] Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 @@ -196,11 +196,11 @@ Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 Input [3]: [s_state#14, _w0#19, s_state#14] Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] -(29) Filter [codegen id : 6] +(29) Filter [codegen id : 3] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Condition : (ranking#20 <= 5) -(30) Project [codegen id : 6] +(30) Project [codegen id : 3] Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] @@ -208,13 +208,13 @@ Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] Input [1]: [s_state#14] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] -(32) BroadcastHashJoin [codegen id : 7] +(32) BroadcastHashJoin [codegen id : 4] Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] Right keys [1]: [s_state#14] Join type: LeftSemi Join condition: None -(33) Project [codegen id : 7] +(33) Project [codegen id : 4] Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#21] Input [3]: [s_store_sk#6, s_county#7, s_state#8] @@ -222,17 +222,17 @@ Input [3]: [s_store_sk#6, s_county#7, s_state#8] Input [3]: [s_store_sk#6, s_county#7, s_state#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(35) BroadcastHashJoin [codegen id : 8] +(35) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#6] Join type: Inner Join condition: None -(36) Project [codegen id : 8] +(36) Project [codegen id : 5] Output [3]: [ss_net_profit#2, s_county#7, s_state#21] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] -(37) HashAggregate [codegen id : 8] +(37) HashAggregate [codegen id : 5] Input [3]: [ss_net_profit#2, s_county#7, s_state#21] Keys [2]: [s_state#21, s_county#7] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] @@ -243,10 +243,10 @@ Results [3]: [s_state#21, s_county#7, sum#23] Input [3]: [s_state#21, s_county#7, sum#23] Arguments: hashpartitioning(s_state#21, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(39) CometColumnarToRow [codegen id : 9] +(39) CometNativeColumnarToRow Input [3]: [s_state#21, s_county#7, sum#23] -(40) HashAggregate [codegen id : 9] +(40) HashAggregate [codegen id : 6] Input [3]: [s_state#21, s_county#7, sum#23] Keys [2]: [s_state#21, s_county#7] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] @@ -256,17 +256,17 @@ Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as d (41) ReusedExchange [Reuses operator id: 38] Output [3]: [s_state#21, s_county#31, sum#32] -(42) CometColumnarToRow [codegen id : 18] +(42) CometNativeColumnarToRow Input [3]: [s_state#21, s_county#31, sum#32] -(43) HashAggregate [codegen id : 18] +(43) HashAggregate [codegen id : 12] Input [3]: [s_state#21, s_county#31, sum#32] Keys [2]: [s_state#21, s_county#31] Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] -(44) HashAggregate [codegen id : 18] +(44) HashAggregate [codegen id : 12] Input [2]: [total_sum#34, s_state#21] Keys [1]: [s_state#21] Functions [1]: [partial_sum(total_sum#34)] @@ -277,10 +277,10 @@ Results [3]: [s_state#21, sum#37, isEmpty#38] Input [3]: [s_state#21, sum#37, isEmpty#38] Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(46) CometColumnarToRow [codegen id : 19] +(46) CometNativeColumnarToRow Input [3]: [s_state#21, sum#37, isEmpty#38] -(47) HashAggregate [codegen id : 19] +(47) HashAggregate [codegen id : 13] Input [3]: [s_state#21, sum#37, isEmpty#38] Keys [1]: [s_state#21] Functions [1]: [sum(total_sum#34)] @@ -290,17 +290,17 @@ Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county (48) ReusedExchange [Reuses operator id: 38] Output [3]: [s_state#21, s_county#45, sum#46] -(49) CometColumnarToRow [codegen id : 28] +(49) CometNativeColumnarToRow Input [3]: [s_state#21, s_county#45, sum#46] -(50) HashAggregate [codegen id : 28] +(50) HashAggregate [codegen id : 19] Input [3]: [s_state#21, s_county#45, sum#46] Keys [2]: [s_state#21, s_county#45] Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#34] -(51) HashAggregate [codegen id : 28] +(51) HashAggregate [codegen id : 19] Input [1]: [total_sum#34] Keys: [] Functions [1]: [partial_sum(total_sum#34)] @@ -311,10 +311,10 @@ Results [2]: [sum#50, isEmpty#51] Input [2]: [sum#50, isEmpty#51] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 29] +(53) CometNativeColumnarToRow Input [2]: [sum#50, isEmpty#51] -(54) HashAggregate [codegen id : 29] +(54) HashAggregate [codegen id : 20] Input [2]: [sum#50, isEmpty#51] Keys: [] Functions [1]: [sum(total_sum#34)] @@ -323,7 +323,7 @@ Results [6]: [sum(total_sum#34)#52 AS total_sum#53, null AS s_state#54, null AS (55) Union -(56) HashAggregate [codegen id : 30] +(56) HashAggregate [codegen id : 21] Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] @@ -347,14 +347,14 @@ Arguments: hashpartitioning(lochierarchy#30, _w0#59, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59], [lochierarchy#30 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(61) CometColumnarToRow [codegen id : 31] +(61) CometNativeColumnarToRow Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] (62) Window Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#59, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#30, _w0#59], [total_sum#25 DESC NULLS LAST] -(63) Project [codegen id : 32] +(63) Project [codegen id : 22] Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59, rank_within_parent#60] @@ -366,7 +366,7 @@ Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometNativeScan parquet spark_catalog.default.date_dim (65) @@ -387,7 +387,7 @@ Condition : (((isnotnull(d_month_seq#61) AND (d_month_seq#61 >= 1212)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#61] Arguments: [d_date_sk#5], [d_date_sk#5] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt index 1a13c28a4f..ed925b771b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -9,7 +9,7 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,19 +21,19 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange @@ -42,7 +42,7 @@ TakeOrderedAndProject : +- Window : +- Sort : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -54,21 +54,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -80,19 +80,19 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange @@ -101,7 +101,7 @@ TakeOrderedAndProject : +- Window : +- Sort : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -113,21 +113,21 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -139,19 +139,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange @@ -160,7 +160,7 @@ TakeOrderedAndProject +- Window +- Sort +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -172,12 +172,12 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/simplified.txt index dcbf0ae4c8..0c358c61ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/simplified.txt @@ -1,106 +1,100 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (32) + WholeStageCodegen (22) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (31) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - 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 (30) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (9) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + 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 (21) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + InputAdapter + CometNativeColumnarToRow + CometFilter [s_store_sk,s_county,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (2) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state] #7 + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #9 - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (13) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_state] #9 + WholeStageCodegen (12) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (20) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (19) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt index 02cdbb3c66..57acc5720f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject (65) : +- BroadcastExchange (35) : +- * Project (34) : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (12) + : :- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) : +- BroadcastExchange (32) @@ -120,7 +120,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -370,7 +370,7 @@ Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) @@ -391,7 +391,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt index a677fed73a..94b23c3b4f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -76,7 +76,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -87,7 +87,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -131,7 +131,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt index 8de8b7e637..808340330f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -46,8 +44,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 02cdbb3c66..57acc5720f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -27,7 +27,7 @@ TakeOrderedAndProject (65) : +- BroadcastExchange (35) : +- * Project (34) : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (12) + : :- CometNativeColumnarToRow (12) : : +- CometFilter (11) : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) : +- BroadcastExchange (32) @@ -120,7 +120,7 @@ ReadSchema: struct Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] +(12) CometNativeColumnarToRow Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -370,7 +370,7 @@ Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (70) -+- * CometColumnarToRow (69) ++- CometNativeColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) @@ -391,7 +391,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(69) CometColumnarToRow [codegen id : 1] +(69) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (70) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt index a677fed73a..94b23c3b4f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,7 +32,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -76,7 +76,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -87,7 +87,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange @@ -131,7 +131,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -142,7 +142,7 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- BroadcastHashJoin - :- CometColumnarToRow + :- CometNativeColumnarToRow : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange 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 8de8b7e637..808340330f 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 @@ -31,12 +31,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -46,8 +44,8 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count WholeStageCodegen (3) Project [s_store_sk,s_county,s_state] BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometFilter [s_store_sk,s_county,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/explain.txt index 44a85d557d..c3517c4663 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (70) +CometNativeColumnarToRow (70) +- CometTakeOrderedAndProject (69) +- CometHashAggregate (68) +- CometExchange (67) @@ -30,38 +30,38 @@ : : : : : : : : : : +- * ColumnarToRow (2) : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) + : : : : : : : : : +- CometNativeColumnarToRow (6) : : : : : : : : : +- CometFilter (5) : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) + : : : : : : : : +- CometNativeColumnarToRow (12) : : : : : : : : +- CometFilter (11) : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) + : : : : : : : +- CometNativeColumnarToRow (18) : : : : : : : +- CometFilter (17) : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) + : : : : : : +- CometNativeColumnarToRow (25) : : : : : : +- CometProject (24) : : : : : : +- CometFilter (23) : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) + : : : : : +- CometNativeColumnarToRow (32) : : : : : +- CometProject (31) : : : : : +- CometFilter (30) : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) : : : : +- ReusedExchange (36) : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) + : : : +- CometNativeColumnarToRow (41) : : : +- CometFilter (40) : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) + : : +- CometNativeColumnarToRow (47) : : +- CometFilter (46) : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometFilter (52) : +- CometNativeScan parquet spark_catalog.default.promotion (51) +- CometSort (63) @@ -79,10 +79,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 10] +(2) ColumnarToRow [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -(3) Filter [codegen id : 10] +(3) Filter [codegen id : 1] Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) @@ -98,20 +98,20 @@ ReadSchema: struct Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Condition : isnotnull(w_warehouse_sk#14) -(12) CometColumnarToRow [codegen id : 2] +(12) CometNativeColumnarToRow Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] (13) BroadcastExchange Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(14) BroadcastHashJoin [codegen id : 10] +(14) BroadcastHashJoin [codegen id : 1] Left keys [1]: [inv_warehouse_sk#11] Right keys [1]: [w_warehouse_sk#14] Join type: Inner Join condition: None -(15) Project [codegen id : 10] +(15) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] @@ -154,20 +154,20 @@ ReadSchema: struct Input [2]: [i_item_sk#16, i_item_desc#17] Condition : isnotnull(i_item_sk#16) -(18) CometColumnarToRow [codegen id : 3] +(18) CometNativeColumnarToRow Input [2]: [i_item_sk#16, i_item_desc#17] (19) BroadcastExchange Input [2]: [i_item_sk#16, i_item_desc#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(20) BroadcastHashJoin [codegen id : 10] +(20) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#4] Right keys [1]: [i_item_sk#16] Join type: Inner Join condition: None -(21) Project [codegen id : 10] +(21) Project [codegen id : 1] Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] @@ -186,20 +186,20 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [cd_demo_sk#18, cd_marital_status#19] Arguments: [cd_demo_sk#18], [cd_demo_sk#18] -(25) CometColumnarToRow [codegen id : 4] +(25) CometNativeColumnarToRow Input [1]: [cd_demo_sk#18] (26) BroadcastExchange Input [1]: [cd_demo_sk#18] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(27) BroadcastHashJoin [codegen id : 10] +(27) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_cdemo_sk#2] Right keys [1]: [cd_demo_sk#18] Join type: Inner Join condition: None -(28) Project [codegen id : 10] +(28) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] @@ -218,33 +218,33 @@ Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharC Input [2]: [hd_demo_sk#20, hd_buy_potential#21] Arguments: [hd_demo_sk#20], [hd_demo_sk#20] -(32) CometColumnarToRow [codegen id : 5] +(32) CometNativeColumnarToRow Input [1]: [hd_demo_sk#20] (33) BroadcastExchange Input [1]: [hd_demo_sk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -(34) BroadcastHashJoin [codegen id : 10] +(34) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_bill_hdemo_sk#3] Right keys [1]: [hd_demo_sk#20] Join type: Inner Join condition: None -(35) Project [codegen id : 10] +(35) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] (36) ReusedExchange [Reuses operator id: 75] Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -(37) BroadcastHashJoin [codegen id : 10] +(37) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#8] Right keys [1]: [d_date_sk#22] Join type: Inner Join condition: None -(38) Project [codegen id : 10] +(38) Project [codegen id : 1] Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] @@ -259,20 +259,20 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_week_seq#26] Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) -(41) CometColumnarToRow [codegen id : 7] +(41) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_week_seq#26] (42) BroadcastExchange Input [2]: [d_date_sk#25, d_week_seq#26] Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] -(43) BroadcastHashJoin [codegen id : 10] +(43) BroadcastHashJoin [codegen id : 1] Left keys [2]: [d_week_seq#24, inv_date_sk#13] Right keys [2]: [d_week_seq#26, d_date_sk#25] Join type: Inner Join condition: None -(44) Project [codegen id : 10] +(44) Project [codegen id : 1] Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] @@ -287,20 +287,20 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_date#28] Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) -(47) CometColumnarToRow [codegen id : 8] +(47) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_date#28] (48) BroadcastExchange Input [2]: [d_date_sk#27, d_date#28] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_ship_date_sk#1] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: (d_date#28 > date_add(d_date#23, 5)) -(50) Project [codegen id : 10] +(50) Project [codegen id : 1] Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] @@ -315,20 +315,20 @@ ReadSchema: struct Input [1]: [p_promo_sk#29] Condition : isnotnull(p_promo_sk#29) -(53) CometColumnarToRow [codegen id : 9] +(53) CometNativeColumnarToRow Input [1]: [p_promo_sk#29] (54) BroadcastExchange Input [1]: [p_promo_sk#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(55) BroadcastHashJoin [codegen id : 10] +(55) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_promo_sk#5] Right keys [1]: [p_promo_sk#29] Join type: LeftOuter Join condition: None -(56) Project [codegen id : 10] +(56) Project [codegen id : 1] Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] @@ -390,14 +390,14 @@ Functions [1]: [count(1)] Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -(70) CometColumnarToRow [codegen id : 11] +(70) CometNativeColumnarToRow Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (75) -+- * CometColumnarToRow (74) ++- CometNativeColumnarToRow (74) +- CometProject (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -418,7 +418,7 @@ Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(74) CometColumnarToRow [codegen id : 1] +(74) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (75) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt index 081972705a..62c2b17b7e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -30,47 +30,47 @@ CometColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/simplified.txt index 025217a607..edcf2b00f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/simplified.txt @@ -1,107 +1,87 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - 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] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] +CometNativeColumnarToRow + CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] + 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] + CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] + CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + CometColumnarExchange [cs_item_sk,cs_order_number] #2 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,inv_date_sk,w_warehouse_name,i_item_desc,d_date,d_week_seq] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Project [cs_ship_date_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name,i_item_desc] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_sold_date_sk,inv_warehouse_sk,inv_date_sk] + BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + Scan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + BroadcastExchange #4 + CometNativeColumnarToRow + CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [w_warehouse_sk,w_warehouse_name] + CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + BroadcastExchange #6 + CometNativeColumnarToRow + CometFilter [i_item_sk,i_item_desc] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + BroadcastExchange #7 + CometNativeColumnarToRow + CometProject [cd_demo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + BroadcastExchange #8 + CometNativeColumnarToRow + CometProject [hd_demo_sk] + CometFilter [hd_demo_sk,hd_buy_potential] + CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + BroadcastExchange #9 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_week_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + BroadcastExchange #10 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + CometNativeColumnarToRow + CometFilter [p_promo_sk] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] + CometSort [cr_item_sk,cr_order_number] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/explain.txt index ee3242f2b1..637427365a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/simplified.txt index 678f8c67df..549469c03b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/simplified.txt @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index ee3242f2b1..637427365a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -371,7 +371,7 @@ Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, pro Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (69) -+- * CometColumnarToRow (68) ++- CometNativeColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] -(68) CometColumnarToRow [codegen id : 1] +(68) CometNativeColumnarToRow Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/extended.txt index 9000d27d1b..819698e99a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/extended.txt @@ -29,7 +29,7 @@ CometColumnarToRow : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometNativeColumnarToRow : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 678f8c67df..549469c03b 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 @@ -31,12 +31,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + CometNativeColumnarToRow + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_date_sk,d_date,d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] CometBroadcastExchange [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] #4 CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/explain.txt index 5d2bcfdc5c..4eacb85276 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/explain.txt @@ -7,14 +7,14 @@ TakeOrderedAndProject (79) : :- * BroadcastHashJoin Inner BuildRight (37) : : :- * Filter (18) : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) + : : : +- CometNativeColumnarToRow (16) : : : +- CometColumnarExchange (15) : : : +- * HashAggregate (14) : : : +- * Project (13) : : : +- * BroadcastHashJoin Inner BuildRight (12) : : : :- * Project (10) : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) + : : : : :- CometNativeColumnarToRow (4) : : : : : +- CometProject (3) : : : : : +- CometFilter (2) : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) @@ -25,14 +25,14 @@ TakeOrderedAndProject (79) : : : +- ReusedExchange (11) : : +- BroadcastExchange (36) : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) + : : +- CometNativeColumnarToRow (34) : : +- CometColumnarExchange (33) : : +- * HashAggregate (32) : : +- * Project (31) : : +- * BroadcastHashJoin Inner BuildRight (30) : : :- * Project (28) : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) + : : : :- CometNativeColumnarToRow (22) : : : : +- CometProject (21) : : : : +- CometFilter (20) : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) @@ -44,14 +44,14 @@ TakeOrderedAndProject (79) : +- BroadcastExchange (56) : +- * Filter (55) : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) + : +- CometNativeColumnarToRow (53) : +- CometColumnarExchange (52) : +- * HashAggregate (51) : +- * Project (50) : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (47) : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) + : : :- CometNativeColumnarToRow (41) : : : +- CometProject (40) : : : +- CometFilter (39) : : : +- CometNativeScan parquet spark_catalog.default.customer (38) @@ -62,14 +62,14 @@ TakeOrderedAndProject (79) : +- ReusedExchange (48) +- BroadcastExchange (76) +- * HashAggregate (75) - +- * CometColumnarToRow (74) + +- CometNativeColumnarToRow (74) +- CometColumnarExchange (73) +- * HashAggregate (72) +- * Project (71) +- * BroadcastHashJoin Inner BuildRight (70) :- * Project (68) : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) + : :- CometNativeColumnarToRow (62) : : +- CometProject (61) : : +- CometFilter (60) : : +- CometNativeScan parquet spark_catalog.default.customer (59) @@ -95,7 +95,7 @@ Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apa Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] -(4) CometColumnarToRow [codegen id : 3] +(4) CometNativeColumnarToRow Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] (5) Scan parquet spark_catalog.default.store_sales @@ -117,30 +117,30 @@ Condition : isnotnull(ss_customer_sk#8) Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#8] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] (11) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#12, d_year#13] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#10] Right keys [1]: [d_date_sk#12] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 2] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 2] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] @@ -151,17 +151,17 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 16] +(16) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -(17) HashAggregate [codegen id : 16] +(17) HashAggregate [codegen id : 12] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] -(18) Filter [codegen id : 16] +(18) Filter [codegen id : 12] Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) @@ -180,7 +180,7 @@ Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] -(22) CometColumnarToRow [codegen id : 6] +(22) CometNativeColumnarToRow Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] (23) Scan parquet spark_catalog.default.store_sales @@ -191,10 +191,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 4] +(24) ColumnarToRow [codegen id : 3] Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -(25) Filter [codegen id : 4] +(25) Filter [codegen id : 3] Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Condition : isnotnull(ss_customer_sk#23) @@ -202,30 +202,30 @@ Condition : isnotnull(ss_customer_sk#23) Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 4] Left keys [1]: [c_customer_sk#19] Right keys [1]: [ss_customer_sk#23] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 4] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#27, d_year#28] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#25] Right keys [1]: [d_date_sk#27] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 4] Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] -(32) HashAggregate [codegen id : 6] +(32) HashAggregate [codegen id : 4] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] @@ -236,10 +236,10 @@ Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(34) CometColumnarToRow [codegen id : 7] +(34) CometNativeColumnarToRow Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -(35) HashAggregate [codegen id : 7] +(35) HashAggregate [codegen id : 5] Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] @@ -250,7 +250,7 @@ Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_firs Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] -(37) BroadcastHashJoin [codegen id : 16] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#31] Join type: Inner @@ -271,7 +271,7 @@ Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] -(41) CometColumnarToRow [codegen id : 10] +(41) CometNativeColumnarToRow Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] (42) Scan parquet spark_catalog.default.web_sales @@ -282,10 +282,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) ColumnarToRow [codegen id : 8] +(43) ColumnarToRow [codegen id : 6] Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(44) Filter [codegen id : 8] +(44) Filter [codegen id : 6] Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Condition : isnotnull(ws_bill_customer_sk#42) @@ -293,30 +293,30 @@ Condition : isnotnull(ws_bill_customer_sk#42) Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -(46) BroadcastHashJoin [codegen id : 10] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [c_customer_sk#35] Right keys [1]: [ws_bill_customer_sk#42] Join type: Inner Join condition: None -(47) Project [codegen id : 10] +(47) Project [codegen id : 7] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) ReusedExchange [Reuses operator id: 83] Output [2]: [d_date_sk#45, d_year#46] -(49) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#44] Right keys [1]: [d_date_sk#45] Join type: Inner Join condition: None -(50) Project [codegen id : 10] +(50) Project [codegen id : 7] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] -(51) HashAggregate [codegen id : 10] +(51) HashAggregate [codegen id : 7] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] @@ -327,17 +327,17 @@ Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum# Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(53) CometColumnarToRow [codegen id : 11] +(53) CometNativeColumnarToRow Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -(54) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 8] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#49] Results [2]: [c_customer_id#39 AS customer_id#50, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#49,17,2) AS year_total#51] -(55) Filter [codegen id : 11] +(55) Filter [codegen id : 8] Input [2]: [customer_id#50, year_total#51] Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) @@ -345,13 +345,13 @@ Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) Input [2]: [customer_id#50, year_total#51] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] -(57) BroadcastHashJoin [codegen id : 16] +(57) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#50] Join type: Inner Join condition: None -(58) Project [codegen id : 16] +(58) Project [codegen id : 12] Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51] Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#50, year_total#51] @@ -370,7 +370,7 @@ Condition : (isnotnull(c_customer_sk#52) AND isnotnull(staticinvoke(class org.ap Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] Arguments: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#54, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#55, 30, true, false, true) AS c_last_name#41] -(62) CometColumnarToRow [codegen id : 14] +(62) CometNativeColumnarToRow Input [4]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41] (63) Scan parquet spark_catalog.default.web_sales @@ -381,10 +381,10 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) ColumnarToRow [codegen id : 12] +(64) ColumnarToRow [codegen id : 9] Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -(65) Filter [codegen id : 12] +(65) Filter [codegen id : 9] Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] Condition : isnotnull(ws_bill_customer_sk#56) @@ -392,30 +392,30 @@ Condition : isnotnull(ws_bill_customer_sk#56) Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(67) BroadcastHashJoin [codegen id : 14] +(67) BroadcastHashJoin [codegen id : 10] Left keys [1]: [c_customer_sk#52] Right keys [1]: [ws_bill_customer_sk#56] Join type: Inner Join condition: None -(68) Project [codegen id : 14] +(68) Project [codegen id : 10] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58] Input [7]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] (69) ReusedExchange [Reuses operator id: 87] Output [2]: [d_date_sk#59, d_year#60] -(70) BroadcastHashJoin [codegen id : 14] +(70) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ws_sold_date_sk#58] Right keys [1]: [d_date_sk#59] Join type: Inner Join condition: None -(71) Project [codegen id : 14] +(71) Project [codegen id : 10] Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58, d_date_sk#59, d_year#60] -(72) HashAggregate [codegen id : 14] +(72) HashAggregate [codegen id : 10] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#57))] @@ -426,10 +426,10 @@ Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum# Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(74) CometColumnarToRow [codegen id : 15] +(74) CometNativeColumnarToRow Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -(75) HashAggregate [codegen id : 15] +(75) HashAggregate [codegen id : 11] Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] Functions [1]: [sum(UnscaledValue(ws_net_paid#57))] @@ -440,13 +440,13 @@ Results [2]: [c_customer_id#39 AS customer_id#63, MakeDecimal(sum(UnscaledValue( Input [2]: [customer_id#63, year_total#64] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] -(77) BroadcastHashJoin [codegen id : 16] +(77) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#17] Right keys [1]: [customer_id#63] Join type: Inner Join condition: (CASE WHEN (year_total#51 > 0.00) THEN (year_total#64 / year_total#51) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) -(78) Project [codegen id : 16] +(78) Project [codegen id : 12] Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51, customer_id#63, year_total#64] @@ -458,7 +458,7 @@ Arguments: 100, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULL Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (83) -+- * CometColumnarToRow (82) ++- CometNativeColumnarToRow (82) +- CometFilter (81) +- CometNativeScan parquet spark_catalog.default.date_dim (80) @@ -474,7 +474,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(82) CometColumnarToRow [codegen id : 1] +(82) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (83) BroadcastExchange @@ -483,7 +483,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (87) -+- * CometColumnarToRow (86) ++- CometNativeColumnarToRow (86) +- CometFilter (85) +- CometNativeScan parquet spark_catalog.default.date_dim (84) @@ -499,7 +499,7 @@ ReadSchema: struct Input [2]: [d_date_sk#27, d_year#28] Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) -(86) CometColumnarToRow [codegen id : 1] +(86) CometNativeColumnarToRow Input [2]: [d_date_sk#27, d_year#28] (87) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt index 0d59f7b833..5d218998be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt @@ -6,14 +6,14 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project : : : +- BroadcastHashJoin : : : :- Project : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow + : : : : :- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -23,23 +23,23 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project : : +- BroadcastHashJoin : : :- Project : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow + : : : :- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -49,24 +49,24 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Filter : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- CometColumnarToRow + : : :- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -76,19 +76,19 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project +- BroadcastHashJoin :- Project : +- BroadcastHashJoin - : :- CometColumnarToRow + : :- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -98,7 +98,7 @@ TakeOrderedAndProject : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : +- ReusedSubquery +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/simplified.txt index 9a8a029e1b..95bb7acab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (16) + WholeStageCodegen (12) Project [customer_id,customer_first_name,customer_last_name] BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] @@ -7,17 +7,17 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] BroadcastHashJoin [customer_id,customer_id] Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] @@ -30,69 +30,65 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #4 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) + WholeStageCodegen (4) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #6 - WholeStageCodegen (4) + WholeStageCodegen (3) Filter [ss_customer_sk] ColumnarToRow InputAdapter Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter ReusedExchange [d_date_sk,d_year] #7 InputAdapter BroadcastExchange #8 - WholeStageCodegen (11) + WholeStageCodegen (8) Filter [year_total] HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) + WholeStageCodegen (7) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #10 - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter @@ -102,25 +98,25 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] ReusedExchange [d_date_sk,d_year] #3 InputAdapter BroadcastExchange #11 - WholeStageCodegen (15) + WholeStageCodegen (11) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) + WholeStageCodegen (10) HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] InputAdapter BroadcastExchange #13 - WholeStageCodegen (12) + WholeStageCodegen (9) Filter [ws_bill_customer_sk] ColumnarToRow InputAdapter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/explain.txt index b0ae8bb695..80326e1f49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_year#26] Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_year#26] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/simplified.txt index 72e39422e0..9ec698fdca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/simplified.txt @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index b0ae8bb695..80326e1f49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -415,7 +415,7 @@ Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (76) -+- * CometColumnarToRow (75) ++- CometNativeColumnarToRow (75) +- CometFilter (74) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) @@ -431,7 +431,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) -(75) CometColumnarToRow [codegen id : 1] +(75) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (76) BroadcastExchange @@ -440,7 +440,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 BroadcastExchange (80) -+- * CometColumnarToRow (79) ++- CometNativeColumnarToRow (79) +- CometFilter (78) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) @@ -456,7 +456,7 @@ ReadSchema: struct Input [2]: [d_date_sk#25, d_year#26] Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) -(79) CometColumnarToRow [codegen id : 1] +(79) CometNativeColumnarToRow Input [2]: [d_date_sk#25, d_year#26] (80) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/extended.txt index 476c7be954..9fb1ccfbe0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,7 +43,7 @@ CometColumnarToRow : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange 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 72e39422e0..9ec698fdca 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 @@ -23,11 +23,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #4 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,11 +45,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [d_date_sk,d_year] #9 CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/explain.txt index eddee4d8e5..1c2b04e6b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (127) +CometNativeColumnarToRow (127) +- CometTakeOrderedAndProject (126) +- CometProject (125) +- CometSortMergeJoin (124) @@ -25,7 +25,7 @@ : : : : : +- * ColumnarToRow (2) : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) + : : : : +- CometNativeColumnarToRow (7) : : : : +- CometProject (6) : : : : +- CometFilter (5) : : : : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -136,10 +136,10 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sol PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Condition : isnotnull(cs_item_sk#1) @@ -158,33 +158,33 @@ Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarc Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (8) BroadcastExchange Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_item_sk#1] Right keys [1]: [i_item_sk#7] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (11) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#13, d_year#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#13] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] @@ -236,36 +236,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(24) ColumnarToRow [codegen id : 6] +(24) ColumnarToRow [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -(25) Filter [codegen id : 6] +(25) Filter [codegen id : 2] Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] Condition : isnotnull(ss_item_sk#22) (26) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -(27) BroadcastHashJoin [codegen id : 6] +(27) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#22] Right keys [1]: [i_item_sk#27] Join type: Inner Join condition: None -(28) Project [codegen id : 6] +(28) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] (29) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#32, d_year#33] -(30) BroadcastHashJoin [codegen id : 6] +(30) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#26] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(31) Project [codegen id : 6] +(31) Project [codegen id : 2] Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] @@ -317,36 +317,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(42) ColumnarToRow [codegen id : 9] +(42) ColumnarToRow [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -(43) Filter [codegen id : 9] +(43) Filter [codegen id : 3] Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Condition : isnotnull(ws_item_sk#41) (44) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -(45) BroadcastHashJoin [codegen id : 9] +(45) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_item_sk#41] Right keys [1]: [i_item_sk#46] Join type: Inner Join condition: None -(46) Project [codegen id : 9] +(46) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] (47) ReusedExchange [Reuses operator id: 131] Output [2]: [d_date_sk#51, d_year#52] -(48) BroadcastHashJoin [codegen id : 9] +(48) BroadcastHashJoin [codegen id : 3] Left keys [1]: [ws_sold_date_sk#45] Right keys [1]: [d_date_sk#51] Join type: Inner Join condition: None -(49) Project [codegen id : 9] +(49) Project [codegen id : 3] Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] @@ -443,36 +443,36 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(70) ColumnarToRow [codegen id : 12] +(70) ColumnarToRow [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -(71) Filter [codegen id : 12] +(71) Filter [codegen id : 4] Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] Condition : isnotnull(cs_item_sk#64) (72) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -(73) BroadcastHashJoin [codegen id : 12] +(73) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_item_sk#64] Right keys [1]: [i_item_sk#70] Join type: Inner Join condition: None -(74) Project [codegen id : 12] +(74) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] (75) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#75, d_year#76] -(76) BroadcastHashJoin [codegen id : 12] +(76) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#68] Right keys [1]: [d_date_sk#75] Join type: Inner Join condition: None -(77) Project [codegen id : 12] +(77) Project [codegen id : 4] Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] @@ -508,36 +508,36 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) ColumnarToRow [codegen id : 15] +(85) ColumnarToRow [codegen id : 5] Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -(86) Filter [codegen id : 15] +(86) Filter [codegen id : 5] Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] Condition : isnotnull(ss_item_sk#81) (87) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] -(88) BroadcastHashJoin [codegen id : 15] +(88) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#81] Right keys [1]: [i_item_sk#86] Join type: Inner Join condition: None -(89) Project [codegen id : 15] +(89) Project [codegen id : 5] Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] Input [10]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] (90) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#91, d_year#92] -(91) BroadcastHashJoin [codegen id : 15] +(91) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#85] Right keys [1]: [d_date_sk#91] Join type: Inner Join condition: None -(92) Project [codegen id : 15] +(92) Project [codegen id : 5] Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] Input [11]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_date_sk#91, d_year#92] @@ -573,36 +573,36 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 18] +(100) ColumnarToRow [codegen id : 6] Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -(101) Filter [codegen id : 18] +(101) Filter [codegen id : 6] Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] Condition : isnotnull(ws_item_sk#97) (102) ReusedExchange [Reuses operator id: 8] Output [5]: [i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] -(103) BroadcastHashJoin [codegen id : 18] +(103) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#97] Right keys [1]: [i_item_sk#102] Join type: Inner Join condition: None -(104) Project [codegen id : 18] +(104) Project [codegen id : 6] Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] Input [10]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] (105) ReusedExchange [Reuses operator id: 135] Output [2]: [d_date_sk#107, d_year#108] -(106) BroadcastHashJoin [codegen id : 18] +(106) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#101] Right keys [1]: [d_date_sk#107] Join type: Inner Join condition: None -(107) Project [codegen id : 18] +(107) Project [codegen id : 6] Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] Input [11]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_date_sk#107, d_year#108] @@ -688,14 +688,14 @@ Arguments: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#1 Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#120 ASC NULLS FIRST,sales_amt_diff#121 ASC NULLS FIRST], output=[prev_year#116,year#117,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#118,curr_yr_cnt#119,sales_cnt_diff#120,sales_amt_diff#121]), [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], 100, 0, [sales_cnt_diff#120 ASC NULLS FIRST, sales_amt_diff#121 ASC NULLS FIRST], [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] -(127) CometColumnarToRow [codegen id : 19] +(127) CometNativeColumnarToRow Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (131) -+- * CometColumnarToRow (130) ++- CometNativeColumnarToRow (130) +- CometFilter (129) +- CometNativeScan parquet spark_catalog.default.date_dim (128) @@ -711,7 +711,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(130) CometColumnarToRow [codegen id : 1] +(130) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (131) BroadcastExchange @@ -724,7 +724,7 @@ Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN d Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 BroadcastExchange (135) -+- * CometColumnarToRow (134) ++- CometNativeColumnarToRow (134) +- CometFilter (133) +- CometNativeScan parquet spark_catalog.default.date_dim (132) @@ -740,7 +740,7 @@ ReadSchema: struct Input [2]: [d_date_sk#75, d_year#76] Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) -(134) CometColumnarToRow [codegen id : 1] +(134) CometNativeColumnarToRow Input [2]: [d_date_sk#75, d_year#76] (135) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt index 9f6954be38..42106a0ffd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometProject +- CometSortMergeJoin @@ -25,16 +25,16 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -55,12 +55,12 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort @@ -81,12 +81,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -117,16 +117,16 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -147,12 +147,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort @@ -173,12 +173,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/simplified.txt index aaf98f4b09..84b015b466 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/simplified.txt @@ -1,173 +1,165 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - 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 [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 [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] +CometNativeColumnarToRow + CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + 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 [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 [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] - 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] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (1) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + BroadcastExchange #6 + CometNativeColumnarToRow + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometExchange [cr_order_number,cr_item_sk] #7 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (2) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometExchange [sr_ticket_number,sr_item_sk] #9 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (3) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometExchange [wr_order_number,wr_item_sk] #11 + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,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] #13 + 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] #14 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] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometSort [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (4) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometSort [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (5) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + CometSortMergeJoin [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometSort [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + CometColumnarExchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (6) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/explain.txt index 6925eb301e..e66f444aba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/simplified.txt index d0520c8b63..0e2370eea4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/simplified.txt @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 6925eb301e..e66f444aba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -683,7 +683,7 @@ Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id# Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * CometColumnarToRow (127) ++- CometNativeColumnarToRow (127) +- CometFilter (126) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct Input [2]: [d_date_sk#13, d_year#14] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) -(127) CometColumnarToRow [codegen id : 1] +(127) CometNativeColumnarToRow Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * CometColumnarToRow (131) ++- CometNativeColumnarToRow (131) +- CometFilter (130) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct Input [2]: [d_date_sk#77, d_year#78] Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) -(131) CometColumnarToRow [codegen id : 1] +(131) CometNativeColumnarToRow Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/extended.txt index 22ccc17641..a3bbeab3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange 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 d0520c8b63..0e2370eea4 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 @@ -26,11 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] @@ -101,11 +99,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 CometBroadcastExchange [d_date_sk,d_year] #18 CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/explain.txt index 8c28d58ce5..7c2cd44b0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/explain.txt @@ -1,19 +1,19 @@ == Physical Plan == -* CometColumnarToRow (110) +CometNativeColumnarToRow (110) +- CometTakeOrderedAndProject (109) +- CometHashAggregate (108) +- CometColumnarExchange (107) +- * HashAggregate (106) +- Union (105) :- * HashAggregate (90) - : +- * CometColumnarToRow (89) + : +- CometNativeColumnarToRow (89) : +- CometColumnarExchange (88) : +- * HashAggregate (87) : +- Union (86) : :- * Project (32) : : +- * BroadcastHashJoin LeftOuter BuildRight (31) : : :- * HashAggregate (16) - : : : +- * CometColumnarToRow (15) + : : : +- CometNativeColumnarToRow (15) : : : +- CometColumnarExchange (14) : : : +- * HashAggregate (13) : : : +- * Project (12) @@ -25,12 +25,12 @@ : : : : : +- Scan parquet spark_catalog.default.store_sales (1) : : : : +- ReusedExchange (4) : : : +- BroadcastExchange (10) - : : : +- * CometColumnarToRow (9) + : : : +- CometNativeColumnarToRow (9) : : : +- CometFilter (8) : : : +- CometNativeScan parquet spark_catalog.default.store (7) : : +- BroadcastExchange (30) : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) + : : +- CometNativeColumnarToRow (28) : : +- CometColumnarExchange (27) : : +- * HashAggregate (26) : : +- * Project (25) @@ -46,7 +46,7 @@ : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) : : :- BroadcastExchange (42) : : : +- * HashAggregate (41) - : : : +- * CometColumnarToRow (40) + : : : +- CometNativeColumnarToRow (40) : : : +- CometColumnarExchange (39) : : : +- * HashAggregate (38) : : : +- * Project (37) @@ -55,7 +55,7 @@ : : : : +- Scan parquet spark_catalog.default.catalog_sales (33) : : : +- ReusedExchange (35) : : +- * HashAggregate (51) - : : +- * CometColumnarToRow (50) + : : +- CometNativeColumnarToRow (50) : : +- CometColumnarExchange (49) : : +- * HashAggregate (48) : : +- * Project (47) @@ -66,7 +66,7 @@ : +- * Project (85) : +- * BroadcastHashJoin LeftOuter BuildRight (84) : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) + : : +- CometNativeColumnarToRow (68) : : +- CometColumnarExchange (67) : : +- * HashAggregate (66) : : +- * Project (65) @@ -78,12 +78,12 @@ : : : : +- Scan parquet spark_catalog.default.web_sales (54) : : : +- ReusedExchange (57) : : +- BroadcastExchange (63) - : : +- * CometColumnarToRow (62) + : : +- CometNativeColumnarToRow (62) : : +- CometFilter (61) : : +- CometNativeScan parquet spark_catalog.default.web_page (60) : +- BroadcastExchange (83) : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) + : +- CometNativeColumnarToRow (81) : +- CometColumnarExchange (80) : +- * HashAggregate (79) : +- * Project (78) @@ -96,18 +96,18 @@ : : +- ReusedExchange (73) : +- ReusedExchange (76) :- * HashAggregate (97) - : +- * CometColumnarToRow (96) + : +- CometNativeColumnarToRow (96) : +- CometColumnarExchange (95) : +- * HashAggregate (94) : +- * HashAggregate (93) - : +- * CometColumnarToRow (92) + : +- CometNativeColumnarToRow (92) : +- ReusedExchange (91) +- * HashAggregate (104) - +- * CometColumnarToRow (103) + +- CometNativeColumnarToRow (103) +- CometColumnarExchange (102) +- * HashAggregate (101) +- * HashAggregate (100) - +- * CometColumnarToRow (99) + +- CometNativeColumnarToRow (99) +- ReusedExchange (98) @@ -119,23 +119,23 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] Condition : isnotnull(ss_store_sk#1) (4) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#6] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#6] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] @@ -150,24 +150,24 @@ ReadSchema: struct Input [1]: [s_store_sk#7] Condition : isnotnull(s_store_sk#7) -(9) CometColumnarToRow [codegen id : 2] +(9) CometNativeColumnarToRow Input [1]: [s_store_sk#7] (10) BroadcastExchange Input [1]: [s_store_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] -(11) BroadcastHashJoin [codegen id : 3] +(11) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_store_sk#1] Right keys [1]: [s_store_sk#7] Join type: Inner Join condition: None -(12) Project [codegen id : 3] +(12) Project [codegen id : 1] Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -(13) HashAggregate [codegen id : 3] +(13) HashAggregate [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] Keys [1]: [s_store_sk#7] Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] @@ -178,10 +178,10 @@ Results [3]: [s_store_sk#7, sum#10, sum#11] Input [3]: [s_store_sk#7, sum#10, sum#11] Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(15) CometColumnarToRow [codegen id : 8] +(15) CometNativeColumnarToRow Input [3]: [s_store_sk#7, sum#10, sum#11] -(16) HashAggregate [codegen id : 8] +(16) HashAggregate [codegen id : 4] Input [3]: [s_store_sk#7, sum#10, sum#11] Keys [1]: [s_store_sk#7] Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] @@ -196,40 +196,40 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(18) ColumnarToRow [codegen id : 6] +(18) ColumnarToRow [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -(19) Filter [codegen id : 6] +(19) Filter [codegen id : 2] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] Condition : isnotnull(sr_store_sk#16) (20) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#20] -(21) BroadcastHashJoin [codegen id : 6] +(21) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_returned_date_sk#19] Right keys [1]: [d_date_sk#20] Join type: Inner Join condition: None -(22) Project [codegen id : 6] +(22) Project [codegen id : 2] Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] (23) ReusedExchange [Reuses operator id: 10] Output [1]: [s_store_sk#21] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 2] Left keys [1]: [sr_store_sk#16] Right keys [1]: [s_store_sk#21] Join type: Inner Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 2] Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -(26) HashAggregate [codegen id : 6] +(26) HashAggregate [codegen id : 2] Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] Keys [1]: [s_store_sk#21] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] @@ -240,10 +240,10 @@ Results [3]: [s_store_sk#21, sum#24, sum#25] Input [3]: [s_store_sk#21, sum#24, sum#25] Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] +(28) CometNativeColumnarToRow Input [3]: [s_store_sk#21, sum#24, sum#25] -(29) HashAggregate [codegen id : 7] +(29) HashAggregate [codegen id : 3] Input [3]: [s_store_sk#21, sum#24, sum#25] Keys [1]: [s_store_sk#21] Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] @@ -254,13 +254,13 @@ Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26 Input [3]: [s_store_sk#21, returns#28, profit_loss#29] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -(31) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_store_sk#7] Right keys [1]: [s_store_sk#21] Join type: LeftOuter Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 4] Output [5]: [store channel AS channel#30, s_store_sk#7 AS id#31, sales#14, coalesce(returns#28, 0.00) AS returns#32, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#33] Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] @@ -271,23 +271,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 10] +(34) ColumnarToRow [codegen id : 5] Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] (35) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#38] -(36) BroadcastHashJoin [codegen id : 10] +(36) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#37] Right keys [1]: [d_date_sk#38] Join type: Inner Join condition: None -(37) Project [codegen id : 10] +(37) Project [codegen id : 5] Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] -(38) HashAggregate [codegen id : 10] +(38) HashAggregate [codegen id : 5] Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] Keys [1]: [cs_call_center_sk#34] Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] @@ -298,10 +298,10 @@ Results [3]: [cs_call_center_sk#34, sum#41, sum#42] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometColumnarToRow [codegen id : 11] +(40) CometNativeColumnarToRow Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -(41) HashAggregate [codegen id : 11] +(41) HashAggregate [codegen id : 6] Input [3]: [cs_call_center_sk#34, sum#41, sum#42] Keys [1]: [cs_call_center_sk#34] Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] @@ -319,23 +319,23 @@ Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] ReadSchema: struct -(44) ColumnarToRow [codegen id : 13] +(44) ColumnarToRow [codegen id : 7] Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] (45) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#50] -(46) BroadcastHashJoin [codegen id : 13] +(46) BroadcastHashJoin [codegen id : 7] Left keys [1]: [cr_returned_date_sk#49] Right keys [1]: [d_date_sk#50] Join type: Inner Join condition: None -(47) Project [codegen id : 13] +(47) Project [codegen id : 7] Output [2]: [cr_return_amount#47, cr_net_loss#48] Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] -(48) HashAggregate [codegen id : 13] +(48) HashAggregate [codegen id : 7] Input [2]: [cr_return_amount#47, cr_net_loss#48] Keys: [] Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] @@ -346,7 +346,7 @@ Results [2]: [sum#53, sum#54] Input [2]: [sum#53, sum#54] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(50) CometColumnarToRow +(50) CometNativeColumnarToRow Input [2]: [sum#53, sum#54] (51) HashAggregate @@ -356,11 +356,11 @@ Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_ne Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] -(52) BroadcastNestedLoopJoin [codegen id : 14] +(52) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(53) Project [codegen id : 14] +(53) Project [codegen id : 8] Output [5]: [catalog channel AS channel#59, cs_call_center_sk#34 AS id#60, sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#61] Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] @@ -372,23 +372,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(55) ColumnarToRow [codegen id : 17] +(55) ColumnarToRow [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -(56) Filter [codegen id : 17] +(56) Filter [codegen id : 9] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] Condition : isnotnull(ws_web_page_sk#62) (57) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#66] -(58) BroadcastHashJoin [codegen id : 17] +(58) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_sold_date_sk#65] Right keys [1]: [d_date_sk#66] Join type: Inner Join condition: None -(59) Project [codegen id : 17] +(59) Project [codegen id : 9] Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] @@ -403,24 +403,24 @@ ReadSchema: struct Input [1]: [wp_web_page_sk#67] Condition : isnotnull(wp_web_page_sk#67) -(62) CometColumnarToRow [codegen id : 16] +(62) CometNativeColumnarToRow Input [1]: [wp_web_page_sk#67] (63) BroadcastExchange Input [1]: [wp_web_page_sk#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -(64) BroadcastHashJoin [codegen id : 17] +(64) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ws_web_page_sk#62] Right keys [1]: [wp_web_page_sk#67] Join type: Inner Join condition: None -(65) Project [codegen id : 17] +(65) Project [codegen id : 9] Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -(66) HashAggregate [codegen id : 17] +(66) HashAggregate [codegen id : 9] Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] Keys [1]: [wp_web_page_sk#67] Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] @@ -431,10 +431,10 @@ Results [3]: [wp_web_page_sk#67, sum#70, sum#71] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(68) CometColumnarToRow [codegen id : 22] +(68) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -(69) HashAggregate [codegen id : 22] +(69) HashAggregate [codegen id : 12] Input [3]: [wp_web_page_sk#67, sum#70, sum#71] Keys [1]: [wp_web_page_sk#67] Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] @@ -449,40 +449,40 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(71) ColumnarToRow [codegen id : 20] +(71) ColumnarToRow [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -(72) Filter [codegen id : 20] +(72) Filter [codegen id : 10] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] Condition : isnotnull(wr_web_page_sk#76) (73) ReusedExchange [Reuses operator id: 115] Output [1]: [d_date_sk#80] -(74) BroadcastHashJoin [codegen id : 20] +(74) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_returned_date_sk#79] Right keys [1]: [d_date_sk#80] Join type: Inner Join condition: None -(75) Project [codegen id : 20] +(75) Project [codegen id : 10] Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] (76) ReusedExchange [Reuses operator id: 63] Output [1]: [wp_web_page_sk#81] -(77) BroadcastHashJoin [codegen id : 20] +(77) BroadcastHashJoin [codegen id : 10] Left keys [1]: [wr_web_page_sk#76] Right keys [1]: [wp_web_page_sk#81] Join type: Inner Join condition: None -(78) Project [codegen id : 20] +(78) Project [codegen id : 10] Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -(79) HashAggregate [codegen id : 20] +(79) HashAggregate [codegen id : 10] Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] Keys [1]: [wp_web_page_sk#81] Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] @@ -493,10 +493,10 @@ Results [3]: [wp_web_page_sk#81, sum#84, sum#85] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(81) CometColumnarToRow [codegen id : 21] +(81) CometNativeColumnarToRow Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -(82) HashAggregate [codegen id : 21] +(82) HashAggregate [codegen id : 11] Input [3]: [wp_web_page_sk#81, sum#84, sum#85] Keys [1]: [wp_web_page_sk#81] Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] @@ -507,19 +507,19 @@ Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77) Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -(84) BroadcastHashJoin [codegen id : 22] +(84) BroadcastHashJoin [codegen id : 12] Left keys [1]: [wp_web_page_sk#67] Right keys [1]: [wp_web_page_sk#81] Join type: LeftOuter Join condition: None -(85) Project [codegen id : 22] +(85) Project [codegen id : 12] Output [5]: [web channel AS channel#90, wp_web_page_sk#67 AS id#91, sales#74, coalesce(returns#88, 0.00) AS returns#92, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#93] Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] (86) Union -(87) HashAggregate [codegen id : 23] +(87) HashAggregate [codegen id : 13] Input [5]: [channel#30, id#31, sales#14, returns#32, profit#33] Keys [2]: [channel#30, id#31] Functions [3]: [partial_sum(sales#14), partial_sum(returns#32), partial_sum(profit#33)] @@ -530,10 +530,10 @@ Results [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] -(89) CometColumnarToRow [codegen id : 24] +(89) CometNativeColumnarToRow Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(90) HashAggregate [codegen id : 24] +(90) HashAggregate [codegen id : 14] Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Keys [2]: [channel#30, id#31] Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] @@ -543,17 +543,17 @@ Results [5]: [channel#30, id#31, cast(sum(sales#14)#106 as decimal(37,2)) AS sal (91) ReusedExchange [Reuses operator id: 88] Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(92) CometColumnarToRow [codegen id : 48] +(92) CometNativeColumnarToRow Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(93) HashAggregate [codegen id : 48] +(93) HashAggregate [codegen id : 28] Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Keys [2]: [channel#30, id#31] Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] Results [4]: [channel#30, sum(sales#14)#106 AS sales#112, sum(returns#32)#107 AS returns#113, sum(profit#33)#108 AS profit#114] -(94) HashAggregate [codegen id : 48] +(94) HashAggregate [codegen id : 28] Input [4]: [channel#30, sales#112, returns#113, profit#114] Keys [1]: [channel#30] Functions [3]: [partial_sum(sales#112), partial_sum(returns#113), partial_sum(profit#114)] @@ -564,10 +564,10 @@ Results [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, i Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(96) CometColumnarToRow [codegen id : 49] +(96) CometNativeColumnarToRow Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -(97) HashAggregate [codegen id : 49] +(97) HashAggregate [codegen id : 29] Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] Keys [1]: [channel#30] Functions [3]: [sum(sales#112), sum(returns#113), sum(profit#114)] @@ -577,17 +577,17 @@ Results [5]: [channel#30, null AS id#130, sum(sales#112)#127 AS sales#131, sum(r (98) ReusedExchange [Reuses operator id: 88] Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(99) CometColumnarToRow [codegen id : 73] +(99) CometNativeColumnarToRow Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -(100) HashAggregate [codegen id : 73] +(100) HashAggregate [codegen id : 43] Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] Keys [2]: [channel#30, id#31] Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] Results [3]: [sum(sales#14)#106 AS sales#112, sum(returns#32)#107 AS returns#113, sum(profit#33)#108 AS profit#114] -(101) HashAggregate [codegen id : 73] +(101) HashAggregate [codegen id : 43] Input [3]: [sales#112, returns#113, profit#114] Keys: [] Functions [3]: [partial_sum(sales#112), partial_sum(returns#113), partial_sum(profit#114)] @@ -598,10 +598,10 @@ Results [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Input [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] -(103) CometColumnarToRow [codegen id : 74] +(103) CometNativeColumnarToRow Input [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -(104) HashAggregate [codegen id : 74] +(104) HashAggregate [codegen id : 44] Input [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Keys: [] Functions [3]: [sum(sales#112), sum(returns#113), sum(profit#114)] @@ -610,7 +610,7 @@ Results [5]: [null AS channel#149, null AS id#150, sum(sales#112)#146 AS sales#1 (105) Union -(106) HashAggregate [codegen id : 75] +(106) HashAggregate [codegen id : 45] Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Functions: [] @@ -630,14 +630,14 @@ Functions: [] Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#30 ASC NULLS FIRST,id#31 ASC NULLS FIRST], output=[channel#30,id#31,sales#109,returns#110,profit#111]), [channel#30, id#31, sales#109, returns#110, profit#111], 100, 0, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30, id#31, sales#109, returns#110, profit#111] -(110) CometColumnarToRow [codegen id : 76] +(110) CometNativeColumnarToRow Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (115) -+- * CometColumnarToRow (114) ++- CometNativeColumnarToRow (114) +- CometProject (113) +- CometFilter (112) +- CometNativeScan parquet spark_catalog.default.date_dim (111) @@ -658,7 +658,7 @@ Condition : (((isnotnull(d_date#154) AND (d_date#154 >= 1998-08-04)) AND (d_date Input [2]: [d_date_sk#6, d_date#154] Arguments: [d_date_sk#6], [d_date_sk#6] -(114) CometColumnarToRow [codegen id : 1] +(114) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (115) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt index 81b485e6b3..178b946ecb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt @@ -1,18 +1,18 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Project : : +- BroadcastHashJoin : : :- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -24,22 +24,22 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -51,19 +51,19 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -72,12 +72,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -86,14 +86,14 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -105,17 +105,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_page : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -127,27 +127,27 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union : :- Project : : +- BroadcastHashJoin : : :- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -159,22 +159,22 @@ CometColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store : : +- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -186,19 +186,19 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange : : : +- HashAggregate - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometColumnarExchange : : : +- HashAggregate : : : +- Project @@ -207,12 +207,12 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -221,14 +221,14 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -240,17 +240,17 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- ReusedSubquery : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_page : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -262,27 +262,27 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union :- Project : +- BroadcastHashJoin : :- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -294,22 +294,22 @@ CometColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -321,19 +321,19 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange : : +- HashAggregate - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometColumnarExchange : : +- HashAggregate : : +- Project @@ -342,12 +342,12 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -356,14 +356,14 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- BroadcastHashJoin :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -375,17 +375,17 @@ CometColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- ReusedSubquery : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page +- BroadcastExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -397,12 +397,12 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- ReusedSubquery : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/simplified.txt index 6f40fcdddf..9a9dde3ef9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/simplified.txt @@ -1,180 +1,172 @@ -WholeStageCodegen (76) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (24) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (45) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (14) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (13) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + Project [s_store_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_store_sk] #3 + WholeStageCodegen (1) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometFilter [s_store_sk] + CometNativeScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [s_store_sk] #7 + WholeStageCodegen (2) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Filter [sr_store_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #9 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 + Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow + InputAdapter + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (8) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow + CometNativeColumnarToRow + CometColumnarExchange [cs_call_center_sk] #9 + WholeStageCodegen (5) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #10 + WholeStageCodegen (7) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (12) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [wp_web_page_sk] #11 + WholeStageCodegen (9) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_web_page_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow + InputAdapter + BroadcastExchange #12 + CometNativeColumnarToRow + CometFilter [wp_web_page_sk] + CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (11) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] InputAdapter - CometColumnarExchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 + CometNativeColumnarToRow + CometColumnarExchange [wp_web_page_sk] #14 + WholeStageCodegen (10) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Filter [wr_web_page_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #15 - WholeStageCodegen (48) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #16 - WholeStageCodegen (73) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [wp_web_page_sk] #12 + WholeStageCodegen (29) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [channel] #15 + WholeStageCodegen (28) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (44) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #16 + WholeStageCodegen (43) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/explain.txt index e36aaa9291..201ba1623c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/explain.txt @@ -580,7 +580,7 @@ Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -601,7 +601,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt index b4318d03e2..40e2c31863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt @@ -23,7 +23,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -257,7 +257,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/simplified.txt index ac3d312ee8..d559f4bdcb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/simplified.txt @@ -33,12 +33,10 @@ WholeStageCodegen (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index e36aaa9291..201ba1623c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -580,7 +580,7 @@ Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (108) -+- * CometColumnarToRow (107) ++- CometNativeColumnarToRow (107) +- CometProject (106) +- CometFilter (105) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) @@ -601,7 +601,7 @@ Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 < Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(107) CometColumnarToRow [codegen id : 1] +(107) CometNativeColumnarToRow Input [1]: [d_date_sk#6] (108) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt index b4318d03e2..40e2c31863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt @@ -23,7 +23,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,7 +140,7 @@ CometColumnarToRow : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometNativeColumnarToRow : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -257,7 +257,7 @@ CometColumnarToRow : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 ac3d312ee8..d559f4bdcb 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 @@ -33,12 +33,10 @@ WholeStageCodegen (22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/explain.txt index dbd053cfac..e227adc462 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (70) +- * Project (69) - +- * CometColumnarToRow (68) + +- CometNativeColumnarToRow (68) +- CometSortMergeJoin (67) :- CometProject (45) : +- CometSortMergeJoin (44) @@ -372,7 +372,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner -(68) CometColumnarToRow [codegen id : 4] +(68) CometNativeColumnarToRow Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] (69) Project [codegen id : 4] @@ -387,7 +387,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (74) -+- * CometColumnarToRow (73) ++- CometNativeColumnarToRow (73) +- CometFilter (72) +- CometNativeScan parquet spark_catalog.default.date_dim (71) @@ -403,7 +403,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(73) CometColumnarToRow [codegen id : 1] +(73) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (74) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt index 65256f2124..af4b7000b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject +- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSortMergeJoin :- CometProject : +- CometSortMergeJoin @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/simplified.txt index 4711669981..e4f8f19efa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/simplified.txt @@ -1,8 +1,8 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] WholeStageCodegen (4) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter + InputAdapter + CometNativeColumnarToRow CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] @@ -24,11 +24,9 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/explain.txt index 82a5357f89..8e3014f399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/simplified.txt index 273db28e77..ef0945f5d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/simplified.txt @@ -21,11 +21,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.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index 82a5357f89..8e3014f399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * CometColumnarToRow (70) ++- CometNativeColumnarToRow (70) +- CometFilter (69) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct Input [2]: [d_date_sk#12, d_year#13] Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) -(70) CometColumnarToRow [codegen id : 1] +(70) CometNativeColumnarToRow Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/extended.txt index 9d5ba53be1..3b721ab107 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometNativeColumnarToRow : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort 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 273db28e77..ef0945f5d6 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 @@ -21,11 +21,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.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometNativeColumnarToRow + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [sr_item_sk,sr_ticket_number] CometExchange [sr_ticket_number,sr_item_sk] #4 CometProject [sr_item_sk,sr_ticket_number] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/explain.txt index 9f6f06d6f6..c942a5abc2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* CometColumnarToRow (119) +CometNativeColumnarToRow (119) +- CometTakeOrderedAndProject (118) +- CometHashAggregate (117) +- CometExchange (116) @@ -653,14 +653,14 @@ Functions: [] Input [5]: [channel#74, id#75, sales#110, returns#111, profit#112] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#74 ASC NULLS FIRST,id#75 ASC NULLS FIRST], output=[channel#74,id#75,sales#110,returns#111,profit#112]), [channel#74, id#75, sales#110, returns#111, profit#112], 100, 0, [channel#74 ASC NULLS FIRST, id#75 ASC NULLS FIRST], [channel#74, id#75, sales#110, returns#111, profit#112] -(119) CometColumnarToRow [codegen id : 10] +(119) CometNativeColumnarToRow Input [5]: [channel#74, id#75, sales#110, returns#111, profit#112] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (124) -+- * CometColumnarToRow (123) ++- CometNativeColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometNativeScan parquet spark_catalog.default.date_dim (120) @@ -681,7 +681,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(123) CometColumnarToRow [codegen id : 1] +(123) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (124) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt index f752943c47..da58e863bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt @@ -1,4 +1,4 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate +- CometExchange @@ -28,7 +28,7 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,7 +160,7 @@ CometColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -292,7 +292,7 @@ CometColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/simplified.txt index cfce9e20a1..37dfa17b65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/simplified.txt @@ -1,137 +1,133 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] +CometNativeColumnarToRow + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometExchange [channel,id,sales,returns,profit] #1 CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - 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] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [cp_catalog_page_id] #11 - 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] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #12 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - 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] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #16 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [s_store_id] #3 + 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] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 + WholeStageCodegen (1) + Filter [ss_store_sk,ss_item_sk,ss_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [s_store_sk,s_store_id] #8 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [cp_catalog_page_id] #11 + 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] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometColumnarExchange [cs_item_sk,cs_order_number] #12 + WholeStageCodegen (2) + Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [web_site_id] #15 + 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] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometColumnarExchange [ws_item_sk,ws_order_number] #16 + WholeStageCodegen (3) + Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + ColumnarToRow + InputAdapter + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #19 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange #20 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt index 6898d458e7..f96edc298a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt @@ -648,7 +648,7 @@ Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -669,7 +669,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt index 0d6844d93c..943d6e8d1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,7 +156,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -285,7 +285,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt index d27ab827e1..dce81e1279 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #6 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index 6898d458e7..f96edc298a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -648,7 +648,7 @@ Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (121) -+- * CometColumnarToRow (120) ++- CometNativeColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) @@ -669,7 +669,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(120) CometColumnarToRow [codegen id : 1] +(120) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (121) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/extended.txt index 0d6844d93c..943d6e8d1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,7 +156,7 @@ CometColumnarToRow : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -285,7 +285,7 @@ CometColumnarToRow : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometNativeColumnarToRow : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 d27ab827e1..dce81e1279 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 @@ -29,12 +29,10 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] CometExchange [sr_item_sk,sr_ticket_number] #6 CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/explain.txt index 358342baed..83ad4f8e4f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/explain.txt @@ -2,7 +2,7 @@ TakeOrderedAndProject (41) +- * Project (40) +- Window (39) - +- * CometColumnarToRow (38) + +- CometNativeColumnarToRow (38) +- CometSort (37) +- CometExchange (36) +- CometHashAggregate (35) @@ -10,7 +10,7 @@ TakeOrderedAndProject (41) +- * HashAggregate (33) +- Union (32) :- * HashAggregate (17) - : +- * CometColumnarToRow (16) + : +- CometNativeColumnarToRow (16) : +- CometColumnarExchange (15) : +- * HashAggregate (14) : +- * Project (13) @@ -22,23 +22,23 @@ TakeOrderedAndProject (41) : : : +- Scan parquet spark_catalog.default.web_sales (1) : : +- ReusedExchange (4) : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) + : +- CometNativeColumnarToRow (10) : +- CometProject (9) : +- CometFilter (8) : +- CometNativeScan parquet spark_catalog.default.item (7) :- * HashAggregate (24) - : +- * CometColumnarToRow (23) + : +- CometNativeColumnarToRow (23) : +- CometColumnarExchange (22) : +- * HashAggregate (21) : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) + : +- CometNativeColumnarToRow (19) : +- ReusedExchange (18) +- * HashAggregate (31) - +- * CometColumnarToRow (30) + +- CometNativeColumnarToRow (30) +- CometColumnarExchange (29) +- * HashAggregate (28) +- * HashAggregate (27) - +- * CometColumnarToRow (26) + +- CometNativeColumnarToRow (26) +- ReusedExchange (25) @@ -50,23 +50,23 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sol PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] Condition : isnotnull(ws_item_sk#1) (4) ReusedExchange [Reuses operator id: 46] Output [1]: [d_date_sk#5] -(5) BroadcastHashJoin [codegen id : 3] +(5) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_sold_date_sk#3] Right keys [1]: [d_date_sk#5] Join type: Inner Join condition: None -(6) Project [codegen id : 3] +(6) Project [codegen id : 1] Output [2]: [ws_item_sk#1, ws_net_paid#2] Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] @@ -85,24 +85,24 @@ Condition : isnotnull(i_item_sk#6) Input [3]: [i_item_sk#6, i_class#7, i_category#8] Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] -(10) CometColumnarToRow [codegen id : 2] +(10) CometNativeColumnarToRow Input [3]: [i_item_sk#6, i_class#9, i_category#10] (11) BroadcastExchange Input [3]: [i_item_sk#6, i_class#9, i_category#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ws_item_sk#1] Right keys [1]: [i_item_sk#6] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [3]: [ws_net_paid#2, i_class#9, i_category#10] Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [3]: [ws_net_paid#2, i_class#9, i_category#10] Keys [2]: [i_category#10, i_class#9] Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] @@ -113,10 +113,10 @@ Results [3]: [i_category#10, i_class#9, sum#12] Input [3]: [i_category#10, i_class#9, sum#12] Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [3]: [i_category#10, i_class#9, sum#12] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [3]: [i_category#10, i_class#9, sum#12] Keys [2]: [i_category#10, i_class#9] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] @@ -126,17 +126,17 @@ Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as dec (18) ReusedExchange [Reuses operator id: 15] Output [3]: [i_category#10, i_class#9, sum#20] -(19) CometColumnarToRow [codegen id : 8] +(19) CometNativeColumnarToRow Input [3]: [i_category#10, i_class#9, sum#20] -(20) HashAggregate [codegen id : 8] +(20) HashAggregate [codegen id : 4] Input [3]: [i_category#10, i_class#9, sum#20] Keys [2]: [i_category#10, i_class#9] Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#13] Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#13,17,2) AS total_sum#22, i_category#10] -(21) HashAggregate [codegen id : 8] +(21) HashAggregate [codegen id : 4] Input [2]: [total_sum#22, i_category#10] Keys [1]: [i_category#10] Functions [1]: [partial_sum(total_sum#22)] @@ -147,10 +147,10 @@ Results [3]: [i_category#10, sum#25, isEmpty#26] Input [3]: [i_category#10, sum#25, isEmpty#26] Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometColumnarToRow [codegen id : 9] +(23) CometNativeColumnarToRow Input [3]: [i_category#10, sum#25, isEmpty#26] -(24) HashAggregate [codegen id : 9] +(24) HashAggregate [codegen id : 5] Input [3]: [i_category#10, sum#25, isEmpty#26] Keys [1]: [i_category#10] Functions [1]: [sum(total_sum#22)] @@ -160,17 +160,17 @@ Results [6]: [sum(total_sum#22)#27 AS total_sum#28, i_category#10, null AS i_cla (25) ReusedExchange [Reuses operator id: 15] Output [3]: [i_category#10, i_class#9, sum#33] -(26) CometColumnarToRow [codegen id : 13] +(26) CometNativeColumnarToRow Input [3]: [i_category#10, i_class#9, sum#33] -(27) HashAggregate [codegen id : 13] +(27) HashAggregate [codegen id : 7] Input [3]: [i_category#10, i_class#9, sum#33] Keys [2]: [i_category#10, i_class#9] Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#13] Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#13,17,2) AS total_sum#22] -(28) HashAggregate [codegen id : 13] +(28) HashAggregate [codegen id : 7] Input [1]: [total_sum#22] Keys: [] Functions [1]: [partial_sum(total_sum#22)] @@ -181,10 +181,10 @@ Results [2]: [sum#37, isEmpty#38] Input [2]: [sum#37, isEmpty#38] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(30) CometColumnarToRow [codegen id : 14] +(30) CometNativeColumnarToRow Input [2]: [sum#37, isEmpty#38] -(31) HashAggregate [codegen id : 14] +(31) HashAggregate [codegen id : 8] Input [2]: [sum#37, isEmpty#38] Keys: [] Functions [1]: [sum(total_sum#22)] @@ -193,7 +193,7 @@ Results [6]: [sum(total_sum#22)#39 AS total_sum#40, null AS i_category#41, null (32) Union -(33) HashAggregate [codegen id : 15] +(33) HashAggregate [codegen id : 9] Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] Functions: [] @@ -217,14 +217,14 @@ Arguments: hashpartitioning(lochierarchy#19, _w0#46, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46] Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46], [lochierarchy#19 ASC NULLS FIRST, _w0#46 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] -(38) CometColumnarToRow [codegen id : 16] +(38) CometNativeColumnarToRow Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46] (39) Window Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46] Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#46, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#47], [lochierarchy#19, _w0#46], [total_sum#14 DESC NULLS LAST] -(40) Project [codegen id : 17] +(40) Project [codegen id : 10] Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#47] Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46, rank_within_parent#47] @@ -236,7 +236,7 @@ Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (46) -+- * CometColumnarToRow (45) ++- CometNativeColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometNativeScan parquet spark_catalog.default.date_dim (42) @@ -257,7 +257,7 @@ Condition : (((isnotnull(d_month_seq#48) AND (d_month_seq#48 >= 1212)) AND (d_mo Input [2]: [d_date_sk#5, d_month_seq#48] Arguments: [d_date_sk#5], [d_date_sk#5] -(45) CometColumnarToRow [codegen id : 1] +(45) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (46) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt index a5b37c422c..d56e25c3af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -9,7 +9,7 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -21,26 +21,26 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item :- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- HashAggregate - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Project @@ -52,26 +52,26 @@ TakeOrderedAndProject : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -83,17 +83,17 @@ TakeOrderedAndProject : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/simplified.txt index 2ccc8c0c39..ad1b79bc92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/simplified.txt @@ -1,71 +1,65 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (17) + WholeStageCodegen (10) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - 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 (15) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + 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 (9) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (2) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category,i_class] #3 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Filter [ws_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #6 - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #7 - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (5) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_category] #6 + WholeStageCodegen (4) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (8) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange #7 + WholeStageCodegen (7) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + CometNativeColumnarToRow + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt index c05e23f926..ccf55fbba8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt @@ -210,7 +210,7 @@ Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -231,7 +231,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt index 46c47555a8..13ff1e34cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt index 11e3c03d80..67ea1dbf75 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt @@ -23,12 +23,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index c05e23f926..ccf55fbba8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -210,7 +210,7 @@ Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (42) -+- * CometColumnarToRow (41) ++- CometNativeColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) @@ -231,7 +231,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(41) CometColumnarToRow [codegen id : 1] +(41) CometNativeColumnarToRow Input [1]: [d_date_sk#5] (42) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt index 46c47555a8..13ff1e34cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,7 +71,7 @@ TakeOrderedAndProject : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 11e3c03d80..67ea1dbf75 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 @@ -23,12 +23,10 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] CometBroadcastExchange [d_date_sk] #5 CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/explain.txt index 646b85d12e..04452de0b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/explain.txt @@ -1,14 +1,14 @@ == Physical Plan == -* CometColumnarToRow (25) +CometNativeColumnarToRow (25) +- CometSort (24) +- CometColumnarExchange (23) +- * Project (22) +- Window (21) - +- * CometColumnarToRow (20) + +- CometNativeColumnarToRow (20) +- CometSort (19) +- CometColumnarExchange (18) +- * HashAggregate (17) - +- * CometColumnarToRow (16) + +- CometNativeColumnarToRow (16) +- CometColumnarExchange (15) +- * HashAggregate (14) +- * Project (13) @@ -19,7 +19,7 @@ : : +- * ColumnarToRow (2) : : +- Scan parquet spark_catalog.default.store_sales (1) : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) + : +- CometNativeColumnarToRow (7) : +- CometProject (6) : +- CometFilter (5) : +- CometNativeScan parquet spark_catalog.default.item (4) @@ -34,10 +34,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -(3) Filter [codegen id : 3] +(3) Filter [codegen id : 1] Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Condition : isnotnull(ss_item_sk#1) @@ -56,37 +56,37 @@ Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCo Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] -(7) CometColumnarToRow [codegen id : 1] +(7) CometNativeColumnarToRow Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (8) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] -(9) BroadcastHashJoin [codegen id : 3] +(9) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#5] Join type: Inner Join condition: None -(10) Project [codegen id : 3] +(10) Project [codegen id : 1] Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] (11) ReusedExchange [Reuses operator id: 30] Output [1]: [d_date_sk#14] -(12) BroadcastHashJoin [codegen id : 3] +(12) BroadcastHashJoin [codegen id : 1] Left keys [1]: [ss_sold_date_sk#3] Right keys [1]: [d_date_sk#14] Join type: Inner Join condition: None -(13) Project [codegen id : 3] +(13) Project [codegen id : 1] Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -(14) HashAggregate [codegen id : 3] +(14) HashAggregate [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] @@ -97,10 +97,10 @@ Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_ Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(16) CometColumnarToRow [codegen id : 4] +(16) CometNativeColumnarToRow Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -(17) HashAggregate [codegen id : 4] +(17) HashAggregate [codegen id : 2] Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] @@ -115,14 +115,14 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 5] +(20) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] (21) Window Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Project [codegen id : 6] +(22) Project [codegen id : 3] Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] @@ -134,14 +134,14 @@ Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 7] +(25) CometNativeColumnarToRow Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometNativeScan parquet spark_catalog.default.date_dim (26) @@ -162,7 +162,7 @@ Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#2 Input [2]: [d_date_sk#14, d_date#22] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt index 9ab057d467..b8d60879d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt @@ -1,13 +1,13 @@ -CometColumnarToRow +CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Project @@ -19,17 +19,17 @@ CometColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange - : +- CometColumnarToRow + : +- CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- CometColumnarToRow + +- CometNativeColumnarToRow +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/simplified.txt index c1ec019e57..efcfa8e8f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/simplified.txt @@ -1,47 +1,39 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] +CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + CometNativeColumnarToRow + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (2) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + InputAdapter + CometNativeColumnarToRow + CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow InputAdapter - ReusedExchange [d_date_sk] #4 + Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + CometNativeColumnarToRow + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt index 96a0ec392e..ee4192a1eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt @@ -142,7 +142,7 @@ Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt index 3dbaf2e346..91e569a85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt index 46191f59cd..7adce04170 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt @@ -23,12 +23,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index 96a0ec392e..ee4192a1eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -142,7 +142,7 @@ Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_pr Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (30) -+- * CometColumnarToRow (29) ++- CometNativeColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(29) CometNativeColumnarToRow Input [1]: [d_date_sk#14] (30) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt index 3dbaf2e346..91e569a85c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim 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 46191f59cd..7adce04170 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 @@ -23,12 +23,10 @@ WholeStageCodegen (3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometNativeColumnarToRow + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index e0a5c43aef..fe5ea77a89 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -30,8 +30,8 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, FromUnixTime, Literal, TruncDate, TruncTimestamp} import org.apache.spark.sql.catalyst.optimizer.SimplifyExtractValueOps -import org.apache.spark.sql.comet.{CometColumnarToRowExec, CometProjectExec} -import org.apache.spark.sql.execution.{InputAdapter, ProjectExec, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.comet.{CometNativeColumnarToRowExec, CometProjectExec} +import org.apache.spark.sql.execution.{ProjectExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -1020,11 +1020,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { val query = sql(s"select cast(id as string) from $table") val (_, cometPlan) = checkSparkAnswerAndOperator(query) val project = cometPlan - .asInstanceOf[WholeStageCodegenExec] - .child - .asInstanceOf[CometColumnarToRowExec] - .child - .asInstanceOf[InputAdapter] + .asInstanceOf[CometNativeColumnarToRowExec] .child .asInstanceOf[CometProjectExec] val id = project.expressions.head diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 1b2373ad71..696a12d4a2 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo, He import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateMode, BloomFilterAggregate} import org.apache.spark.sql.comet._ import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} -import org.apache.spark.sql.execution.{CollectLimitExec, ProjectExec, SQLExecution, UnionExec} +import org.apache.spark.sql.execution.{CollectLimitExec, ProjectExec, SparkPlan, SQLExecution, UnionExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} @@ -864,9 +864,11 @@ class CometExecSuite extends CometTestBase { checkSparkAnswerAndOperator(df) // Before AQE: one CometBroadcastExchange, no CometColumnarToRow - var columnarToRowExec = stripAQEPlan(df.queryExecution.executedPlan).collect { - case s: CometColumnarToRowExec => s - } + var columnarToRowExec: Seq[SparkPlan] = + stripAQEPlan(df.queryExecution.executedPlan).collect { + case s: CometColumnarToRowExec => s + case s: CometNativeColumnarToRowExec => s + } assert(columnarToRowExec.isEmpty) // Disable CometExecRule after the initial plan is generated. The CometSortMergeJoin and @@ -880,14 +882,25 @@ class CometExecSuite extends CometTestBase { // After AQE: CometBroadcastExchange has to be converted to rows to conform to Spark // BroadcastHashJoin. val plan = stripAQEPlan(df.queryExecution.executedPlan) - columnarToRowExec = plan.collect { case s: CometColumnarToRowExec => - s + columnarToRowExec = plan.collect { + case s: CometColumnarToRowExec => s + case s: CometNativeColumnarToRowExec => s } assert(columnarToRowExec.length == 1) - // This ColumnarToRowExec should be the immediate child of BroadcastHashJoinExec - val parent = plan.find(_.children.contains(columnarToRowExec.head)) - assert(parent.get.isInstanceOf[BroadcastHashJoinExec]) + // This ColumnarToRowExec should be a descendant of BroadcastHashJoinExec (possibly + // wrapped by InputAdapter for codegen). + val broadcastJoins = plan.collect { case b: BroadcastHashJoinExec => b } + assert(broadcastJoins.nonEmpty, s"Expected BroadcastHashJoinExec in plan:\n$plan") + val hasC2RDescendant = broadcastJoins.exists { join => + join.find { + case _: CometColumnarToRowExec | _: CometNativeColumnarToRowExec => true + case _ => false + }.isDefined + } + assert( + hasC2RDescendant, + "BroadcastHashJoinExec should have a columnar-to-row descendant") // There should be a CometBroadcastExchangeExec under CometColumnarToRowExec val broadcastQueryStage = diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeColumnarToRowSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeColumnarToRowSuite.scala new file mode 100644 index 0000000000..c59bcda456 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeColumnarToRowSuite.scala @@ -0,0 +1,484 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.exec + +import java.sql.Date +import java.sql.Timestamp + +import scala.util.Random + +import org.scalactic.source.Position +import org.scalatest.Tag + +import org.apache.spark.sql.{CometTestBase, Row} +import org.apache.spark.sql.comet.CometNativeColumnarToRowExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.types._ + +import org.apache.comet.CometConf +import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, SchemaGenOptions} + +/** + * Test suite for native columnar to row conversion. + * + * These tests verify that CometNativeColumnarToRowExec produces correct results for all supported + * data types. + */ +class CometNativeColumnarToRowSuite extends CometTestBase with AdaptiveSparkPlanHelper { + + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + super.test(testName, testTags: _*) { + withSQLConf( + CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.key -> "true", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { + testFun + } + } + } + + /** + * Helper to verify that CometNativeColumnarToRowExec is present in the plan. + */ + private def assertNativeC2RPresent(df: org.apache.spark.sql.DataFrame): Unit = { + val plan = stripAQEPlan(df.queryExecution.executedPlan) + val nativeC2R = plan.collect { case c: CometNativeColumnarToRowExec => c } + assert( + nativeC2R.nonEmpty, + s"Expected CometNativeColumnarToRowExec in plan but found none.\nPlan: $plan") + } + + test("primitive types: boolean, byte, short, int, long") { + val data = (0 until 100).map { i => + (i % 2 == 0, i.toByte, i.toShort, i, i.toLong) + } + withParquetTable(data, "primitives") { + // Force row output by using a UDF or collect + val df = sql("SELECT * FROM primitives") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("primitive types: float, double") { + val data = (0 until 100).map { i => + (i.toFloat / 10.0f, i.toDouble / 10.0) + } + withParquetTable(data, "floats") { + val df = sql("SELECT * FROM floats") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("primitive types with nulls") { + val data = (0 until 100).map { i => + val isNull = i % 5 == 0 + ( + if (isNull) null else (i % 2 == 0), + if (isNull) null else i.toByte, + if (isNull) null else i.toShort, + if (isNull) null else i, + if (isNull) null else i.toLong, + if (isNull) null else i.toFloat, + if (isNull) null else i.toDouble) + } + val df = spark + .createDataFrame( + spark.sparkContext.parallelize(data.map(Row.fromTuple(_))), + StructType( + Seq( + StructField("bool", BooleanType, nullable = true), + StructField("byte", ByteType, nullable = true), + StructField("short", ShortType, nullable = true), + StructField("int", IntegerType, nullable = true), + StructField("long", LongType, nullable = true), + StructField("float", FloatType, nullable = true), + StructField("double", DoubleType, nullable = true)))) + withParquetDataFrame(df) { parquetDf => + assertNativeC2RPresent(parquetDf) + checkSparkAnswer(parquetDf) + } + } + + test("string type") { + val data = (0 until 100).map { i => + (i, s"string_value_$i", if (i % 10 == 0) null else s"nullable_$i") + } + val df = spark + .createDataFrame( + spark.sparkContext.parallelize(data.map(Row.fromTuple(_))), + StructType( + Seq( + StructField("id", IntegerType), + StructField("str", StringType), + StructField("nullable_str", StringType, nullable = true)))) + withParquetDataFrame(df) { parquetDf => + assertNativeC2RPresent(parquetDf) + checkSparkAnswer(parquetDf) + } + } + + test("string type with various lengths") { + val random = new Random(42) + val data = (0 until 100).map { i => + val len = random.nextInt(1000) + (i, random.alphanumeric.take(len).mkString) + } + withParquetTable(data, "strings") { + val df = sql("SELECT * FROM strings") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("binary type") { + val data = (0 until 100).map { i => + (i, s"binary_$i".getBytes) + } + val df = spark + .createDataFrame( + spark.sparkContext.parallelize(data.map { case (id, bytes) => Row(id, bytes) }), + StructType( + Seq(StructField("id", IntegerType), StructField("bin", BinaryType, nullable = true)))) + withParquetDataFrame(df) { parquetDf => + assertNativeC2RPresent(parquetDf) + checkSparkAnswer(parquetDf) + } + } + + test("date type") { + val baseDate = Date.valueOf("2024-01-01") + val data = (0 until 100).map { i => + (i, new Date(baseDate.getTime + i * 24 * 60 * 60 * 1000L)) + } + withParquetTable(data, "dates") { + val df = sql("SELECT * FROM dates") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("timestamp type") { + val baseTs = Timestamp.valueOf("2024-01-01 00:00:00") + val data = (0 until 100).map { i => + (i, new Timestamp(baseTs.getTime + i * 1000L)) + } + withParquetTable(data, "timestamps") { + val df = sql("SELECT * FROM timestamps") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("decimal type - inline precision (precision <= 18)") { + val data = (0 until 100).map { i => + (i, BigDecimal(i * 100 + i) / 100) + } + val df = spark + .createDataFrame( + spark.sparkContext.parallelize(data.map { case (id, dec) => + Row(id, dec.bigDecimal) + }), + StructType( + Seq( + StructField("id", IntegerType), + StructField("dec", DecimalType(10, 2), nullable = true)))) + withParquetDataFrame(df) { parquetDf => + assertNativeC2RPresent(parquetDf) + checkSparkAnswer(parquetDf) + } + } + + test("decimal type - variable length precision (precision > 18)") { + val data = (0 until 100).map { i => + (i, BigDecimal(s"12345678901234567890.$i")) + } + val df = spark + .createDataFrame( + spark.sparkContext.parallelize(data.map { case (id, dec) => + Row(id, dec.bigDecimal) + }), + StructType( + Seq( + StructField("id", IntegerType), + StructField("dec", DecimalType(30, 5), nullable = true)))) + withParquetDataFrame(df) { parquetDf => + assertNativeC2RPresent(parquetDf) + checkSparkAnswer(parquetDf) + } + } + + test("struct type") { + val data = (0 until 100).map { i => + (i, (i * 2, s"nested_$i")) + } + withParquetTable(data, "structs") { + val df = sql("SELECT * FROM structs") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("array type") { + val data = (0 until 100).map { i => + (i, (0 to i % 10).toArray) + } + withParquetTable(data, "arrays") { + val df = sql("SELECT * FROM arrays") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("array of strings") { + val data = (0 until 100).map { i => + (i, (0 to i % 5).map(j => s"elem_${i}_$j").toArray) + } + withParquetTable(data, "string_arrays") { + val df = sql("SELECT * FROM string_arrays") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("map type") { + val data = (0 until 100).map { i => + (i, (0 to i % 5).map(j => (s"key_$j", j * 10)).toMap) + } + withParquetTable(data, "maps") { + val df = sql("SELECT * FROM maps") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("nested struct in array") { + val data = (0 until 100).map { i => + (i, (0 to i % 5).map(j => (j, s"nested_$j")).toArray) + } + withParquetTable(data, "nested_structs") { + val df = sql("SELECT * FROM nested_structs") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("deeply nested: array of arrays") { + val data = (0 until 100).map { i => + (i, (0 to i % 3).map(j => (0 to j).toArray).toArray) + } + withParquetTable(data, "nested_arrays") { + val df = sql("SELECT * FROM nested_arrays") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("deeply nested: map with array values") { + val data = (0 until 100).map { i => + (i, (0 to i % 3).map(j => (s"key_$j", (0 to j).toArray)).toMap) + } + withParquetTable(data, "map_array_values") { + val df = sql("SELECT * FROM map_array_values") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("deeply nested: struct containing array of maps") { + val data = (0 until 100).map { i => + ( + i, + ((0 to i % 3).map(j => (0 to j % 2).map(k => (s"k$k", k * 10)).toMap).toArray, s"str_$i")) + } + withParquetTable(data, "struct_array_maps") { + val df = sql("SELECT * FROM struct_array_maps") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("all null values") { + val df = spark + .createDataFrame( + spark.sparkContext.parallelize((0 until 100).map(_ => Row(null, null, null))), + StructType( + Seq( + StructField("int_col", IntegerType, nullable = true), + StructField("str_col", StringType, nullable = true), + StructField("double_col", DoubleType, nullable = true)))) + withParquetDataFrame(df) { parquetDf => + assertNativeC2RPresent(parquetDf) + checkSparkAnswer(parquetDf) + } + } + + test("empty batch") { + val df = spark + .createDataFrame( + spark.sparkContext.parallelize(Seq.empty[Row]), + StructType(Seq(StructField("int_col", IntegerType), StructField("str_col", StringType)))) + withParquetDataFrame(df) { parquetDf => + assertNativeC2RPresent(parquetDf) + checkSparkAnswer(parquetDf) + } + } + + test("mixed types - comprehensive") { + val baseDate = Date.valueOf("2024-01-01") + val baseTs = Timestamp.valueOf("2024-01-01 00:00:00") + + val data = (0 until 100).map { i => + val isNull = i % 7 == 0 + Row( + i, + if (isNull) null else (i % 2 == 0), + if (isNull) null else i.toByte, + if (isNull) null else i.toShort, + if (isNull) null else i.toLong, + if (isNull) null else i.toFloat, + if (isNull) null else i.toDouble, + if (isNull) null else s"string_$i", + if (isNull) null else new Date(baseDate.getTime + i * 24 * 60 * 60 * 1000L), + if (isNull) null else new Timestamp(baseTs.getTime + i * 1000L), + if (isNull) null else BigDecimal(i * 100 + i, 2).bigDecimal) + } + + val schema = StructType( + Seq( + StructField("id", IntegerType), + StructField("bool_col", BooleanType, nullable = true), + StructField("byte_col", ByteType, nullable = true), + StructField("short_col", ShortType, nullable = true), + StructField("long_col", LongType, nullable = true), + StructField("float_col", FloatType, nullable = true), + StructField("double_col", DoubleType, nullable = true), + StructField("string_col", StringType, nullable = true), + StructField("date_col", DateType, nullable = true), + StructField("timestamp_col", TimestampType, nullable = true), + StructField("decimal_col", DecimalType(10, 2), nullable = true))) + + val df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + withParquetDataFrame(df) { parquetDf => + assertNativeC2RPresent(parquetDf) + checkSparkAnswer(parquetDf) + } + } + + test("large batch") { + val data = (0 until 10000).map { i => + (i, s"string_value_$i", i.toDouble) + } + withParquetTable(data, "large_batch") { + val df = sql("SELECT * FROM large_batch") + assertNativeC2RPresent(df) + checkSparkAnswer(df) + } + } + + test("disabled by default") { + withSQLConf(CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.key -> "false") { + val data = (0 until 10).map(i => (i, s"value_$i")) + withParquetTable(data, "test_disabled") { + val df = sql("SELECT * FROM test_disabled") + val plan = stripAQEPlan(df.queryExecution.executedPlan) + val nativeC2R = plan.collect { case c: CometNativeColumnarToRowExec => c } + assert( + nativeC2R.isEmpty, + s"Expected no CometNativeColumnarToRowExec when disabled.\nPlan: $plan") + checkSparkAnswer(df) + } + } + } + + test("fuzz test with nested types") { + val random = new Random(42) + val schemaGenOptions = + SchemaGenOptions(generateArray = true, generateStruct = true, generateMap = true) + val dataGenOptions = + DataGenOptions(generateNegativeZero = false, generateNaN = false, generateInfinity = false) + + // Use generateSchema which creates various nested types including arrays, structs, and maps. + // Not all generated types may be supported by native C2R, so we just verify correctness. + val schema = FuzzDataGenerator.generateSchema(schemaGenOptions) + val df = FuzzDataGenerator.generateDataFrame(random, spark, schema, 100, dataGenOptions) + + withParquetDataFrame(df) { parquetDf => + checkSparkAnswer(parquetDf) + } + } + + test("fuzz test with generateNestedSchema") { + val random = new Random(42) + + // Use only primitive types supported by native C2R (excludes TimestampNTZType) + val supportedPrimitiveTypes: Seq[DataType] = Seq( + DataTypes.BooleanType, + DataTypes.ByteType, + DataTypes.ShortType, + DataTypes.IntegerType, + DataTypes.LongType, + DataTypes.FloatType, + DataTypes.DoubleType, + DataTypes.createDecimalType(10, 2), + DataTypes.DateType, + DataTypes.TimestampType, + DataTypes.StringType, + DataTypes.BinaryType) + + val schemaGenOptions = SchemaGenOptions( + generateArray = true, + generateStruct = true, + generateMap = true, + primitiveTypes = supportedPrimitiveTypes) + val dataGenOptions = + DataGenOptions(generateNegativeZero = false, generateNaN = false, generateInfinity = false) + + // Test with multiple random deeply nested schemas + for (iteration <- 1 to 3) { + val schema = FuzzDataGenerator.generateNestedSchema( + random, + numCols = 5, + minDepth = 1, + maxDepth = 3, + options = schemaGenOptions) + + val df = FuzzDataGenerator.generateDataFrame(random, spark, schema, 100, dataGenOptions) + + withParquetDataFrame(df) { parquetDf => + assertNativeC2RPresent(parquetDf) + checkSparkAnswer(parquetDf) + } + } + } + + /** + * Helper to create a parquet table from a DataFrame and run a function with it. + */ + private def withParquetDataFrame(df: org.apache.spark.sql.DataFrame)( + f: org.apache.spark.sql.DataFrame => Unit): Unit = { + withTempPath { path => + df.write.parquet(path.getAbsolutePath) + spark.read.parquet(path.getAbsolutePath).createOrReplaceTempView("test_table") + f(sql("SELECT * FROM test_table")) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometColumnarToRowBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometColumnarToRowBenchmark.scala new file mode 100644 index 0000000000..bcf5d73636 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometColumnarToRowBenchmark.scala @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.benchmark + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.internal.SQLConf + +import org.apache.comet.{CometConf, CometSparkSessionExtensions} + +/** + * Benchmark to compare Columnar to Row conversion performance: + * - Spark's default ColumnarToRowExec + * - Comet's JVM-based CometColumnarToRowExec + * - Comet's Native CometNativeColumnarToRowExec + * + * To run this benchmark: + * {{{ + * SPARK_GENERATE_BENCHMARK_FILES=1 make benchmark-org.apache.spark.sql.benchmark.CometColumnarToRowBenchmark + * }}} + * + * Results will be written to "spark/benchmarks/CometColumnarToRowBenchmark-**results.txt". + */ +object CometColumnarToRowBenchmark extends CometBenchmarkBase { + override def getSparkSession: SparkSession = { + val conf = new SparkConf() + .setAppName("CometColumnarToRowBenchmark") + .set("spark.master", "local[1]") + .setIfMissing("spark.driver.memory", "3g") + .setIfMissing("spark.executor.memory", "3g") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "2g") + + val sparkSession = SparkSession.builder + .config(conf) + .withExtensions(new CometSparkSessionExtensions) + .getOrCreate() + + // Set default configs + sparkSession.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + sparkSession.conf.set(CometConf.COMET_ENABLED.key, "false") + sparkSession.conf.set(CometConf.COMET_EXEC_ENABLED.key, "false") + // Disable dictionary encoding to ensure consistent data representation + sparkSession.conf.set("parquet.enable.dictionary", "false") + + sparkSession + } + + /** + * Helper method to add the standard benchmark cases for columnar to row conversion. Reduces + * code duplication across benchmark methods. + */ + private def addC2RBenchmarkCases(benchmark: Benchmark, query: String): Unit = { + benchmark.addCase("Spark (ColumnarToRowExec)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + spark.sql(query).noop() + } + } + + benchmark.addCase("Comet JVM (CometColumnarToRowExec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.key -> "false") { + spark.sql(query).noop() + } + } + + benchmark.addCase("Comet Native (CometNativeColumnarToRowExec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + } + + /** + * Benchmark columnar to row conversion for primitive types. + */ + def primitiveTypesBenchmark(values: Int): Unit = { + val benchmark = + new Benchmark("Columnar to Row - Primitive Types", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + // Create a table with various primitive types (includes strings) + val df = spark + .range(values) + .selectExpr( + "id as long_col", + "cast(id as int) as int_col", + "cast(id as short) as short_col", + "cast(id as byte) as byte_col", + "cast(id % 2 as boolean) as bool_col", + "cast(id as float) as float_col", + "cast(id as double) as double_col", + "cast(id as string) as string_col", + "date_add(to_date('2024-01-01'), cast(id % 365 as int)) as date_col") + + prepareTable(dir, df) + val query = "SELECT * FROM parquetV1Table" + addC2RBenchmarkCases(benchmark, query) + benchmark.run() + } + } + } + + /** + * Benchmark columnar to row conversion for fixed-width types ONLY (no strings). This tests the + * fast path in native C2R that pre-allocates buffers. + */ + def fixedWidthOnlyBenchmark(values: Int): Unit = { + val benchmark = + new Benchmark("Columnar to Row - Fixed Width Only (no strings)", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + // Create a table with ONLY fixed-width primitive types (no strings!) + val df = spark + .range(values) + .selectExpr( + "id as long_col", + "cast(id as int) as int_col", + "cast(id as short) as short_col", + "cast(id as byte) as byte_col", + "cast(id % 2 as boolean) as bool_col", + "cast(id as float) as float_col", + "cast(id as double) as double_col", + "date_add(to_date('2024-01-01'), cast(id % 365 as int)) as date_col", + "cast(id * 2 as long) as long_col2", + "cast(id * 3 as int) as int_col2") + + prepareTable(dir, df) + val query = "SELECT * FROM parquetV1Table" + addC2RBenchmarkCases(benchmark, query) + benchmark.run() + } + } + } + + /** + * Benchmark columnar to row conversion for string-heavy data. + */ + def stringTypesBenchmark(values: Int): Unit = { + val benchmark = + new Benchmark("Columnar to Row - String Types", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + val df = spark + .range(values) + .selectExpr( + "id", + "concat('short_', cast(id % 100 as string)) as short_str", + "concat('medium_string_value_', cast(id as string), '_with_more_content') as medium_str", + "repeat(concat('long_', cast(id as string)), 10) as long_str") + + prepareTable(dir, df) + val query = "SELECT * FROM parquetV1Table" + addC2RBenchmarkCases(benchmark, query) + benchmark.run() + } + } + } + + /** + * Benchmark columnar to row conversion for nested struct types. + */ + def structTypesBenchmark(values: Int): Unit = { + val benchmark = + new Benchmark("Columnar to Row - Struct Types", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + val df = spark + .range(values) + .selectExpr( + "id", + // Simple struct + "named_struct('a', cast(id as int), 'b', cast(id as string)) as simple_struct", + // Nested struct (2 levels) + """named_struct( + 'outer_int', cast(id as int), + 'inner', named_struct('x', cast(id as double), 'y', cast(id as string)) + ) as nested_struct""", + // Deeply nested struct (3 levels) + """named_struct( + 'level1', named_struct( + 'level2', named_struct( + 'value', cast(id as int), + 'name', concat('item_', cast(id as string)) + ), + 'count', cast(id % 100 as int) + ), + 'id', id + ) as deep_struct""") + + prepareTable(dir, df) + val query = "SELECT * FROM parquetV1Table" + addC2RBenchmarkCases(benchmark, query) + benchmark.run() + } + } + } + + /** + * Benchmark columnar to row conversion for array types. + */ + def arrayTypesBenchmark(values: Int): Unit = { + val benchmark = + new Benchmark("Columnar to Row - Array Types", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + val df = spark + .range(values) + .selectExpr( + "id", + // Array of primitives + "array(cast(id as int), cast(id + 1 as int), cast(id + 2 as int)) as int_array", + // Array of strings + "array(concat('a_', cast(id as string)), concat('b_', cast(id as string))) as str_array", + // Longer array + """array( + cast(id % 10 as int), cast((id + 1) % 10 as int), cast((id + 2) % 10 as int), + cast((id + 3) % 10 as int), cast((id + 4) % 10 as int) + ) as longer_array""") + + prepareTable(dir, df) + val query = "SELECT * FROM parquetV1Table" + addC2RBenchmarkCases(benchmark, query) + benchmark.run() + } + } + } + + /** + * Benchmark columnar to row conversion for map types. + */ + def mapTypesBenchmark(values: Int): Unit = { + val benchmark = + new Benchmark("Columnar to Row - Map Types", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + val df = spark + .range(values) + .selectExpr( + "id", + // Map with string keys and int values + "map('key1', cast(id as int), 'key2', cast(id + 1 as int)) as str_int_map", + // Map with int keys and string values + "map(cast(id % 10 as int), concat('val_', cast(id as string))) as int_str_map", + // Larger map + """map( + 'a', cast(id as double), + 'b', cast(id + 1 as double), + 'c', cast(id + 2 as double) + ) as larger_map""") + + prepareTable(dir, df) + val query = "SELECT * FROM parquetV1Table" + addC2RBenchmarkCases(benchmark, query) + benchmark.run() + } + } + } + + /** + * Benchmark columnar to row conversion for complex nested types (arrays of structs, maps with + * array values, etc.) + */ + def complexNestedTypesBenchmark(values: Int): Unit = { + val benchmark = + new Benchmark("Columnar to Row - Complex Nested Types", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + val df = spark + .range(values) + .selectExpr( + "id", + // Array of structs + """array( + named_struct('id', cast(id as int), 'name', concat('item_', cast(id as string))), + named_struct('id', cast(id + 1 as int), 'name', concat('item_', cast(id + 1 as string))) + ) as array_of_structs""", + // Struct with array field + """named_struct( + 'values', array(cast(id as int), cast(id + 1 as int), cast(id + 2 as int)), + 'label', concat('label_', cast(id as string)) + ) as struct_with_array""", + // Map with array values + """map( + 'scores', array(cast(id % 100 as double), cast((id + 1) % 100 as double)), + 'ranks', array(cast(id % 10 as double)) + ) as map_with_arrays""") + + prepareTable(dir, df) + val query = "SELECT * FROM parquetV1Table" + addC2RBenchmarkCases(benchmark, query) + benchmark.run() + } + } + } + + /** + * Benchmark with wide rows (many columns) to stress test row conversion. + */ + def wideRowsBenchmark(values: Int): Unit = { + val benchmark = + new Benchmark("Columnar to Row - Wide Rows (50 columns)", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + // Generate 50 columns of mixed types + val columns = (0 until 50).map { i => + i % 5 match { + case 0 => s"cast(id + $i as int) as int_col_$i" + case 1 => s"cast(id + $i as long) as long_col_$i" + case 2 => s"cast(id + $i as double) as double_col_$i" + case 3 => s"concat('str_${i}_', cast(id as string)) as str_col_$i" + case 4 => s"cast((id + $i) % 2 as boolean) as bool_col_$i" + } + } + + val df = spark.range(values).selectExpr(columns: _*) + + prepareTable(dir, df) + val query = "SELECT * FROM parquetV1Table" + addC2RBenchmarkCases(benchmark, query) + benchmark.run() + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + val numRows = 1024 * 1024 // 1M rows + + runBenchmark("Columnar to Row Conversion - Fixed Width Only") { + fixedWidthOnlyBenchmark(numRows) + } + + runBenchmark("Columnar to Row Conversion - Primitive Types") { + primitiveTypesBenchmark(numRows) + } + + runBenchmark("Columnar to Row Conversion - String Types") { + stringTypesBenchmark(numRows) + } + + runBenchmark("Columnar to Row Conversion - Struct Types") { + structTypesBenchmark(numRows) + } + + runBenchmark("Columnar to Row Conversion - Array Types") { + arrayTypesBenchmark(numRows) + } + + runBenchmark("Columnar to Row Conversion - Map Types") { + mapTypesBenchmark(numRows) + } + + runBenchmark("Columnar to Row Conversion - Complex Nested Types") { + complexNestedTypesBenchmark(numRows) + } + + runBenchmark("Columnar to Row Conversion - Wide Rows") { + wideRowsBenchmark(numRows) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanChecker.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanChecker.scala index 7caac71351..c8c4baff4a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanChecker.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanChecker.scala @@ -46,7 +46,7 @@ trait CometPlanChecker { case _: CometNativeScanExec | _: CometScanExec | _: CometBatchScanExec | _: CometIcebergNativeScanExec => case _: CometSinkPlaceHolder | _: CometScanWrapper => - case _: CometColumnarToRowExec => + case _: CometColumnarToRowExec | _: CometNativeColumnarToRowExec => case _: CometSparkToColumnarExec => case _: CometExec | _: CometShuffleExchangeExec => case _: CometBroadcastExchangeExec =>